From f3c0f19fb204a18192f2a78fe72378fd292d4641 Mon Sep 17 00:00:00 2001 From: Dan Rammer Date: Fri, 12 Apr 2024 15:30:10 -0500 Subject: [PATCH] execution environments (#164) * added ExecutionEnvironment protos Signed-off-by: Daniel Rammer * updated naming Signed-off-by: Daniel Rammer * Agents protocol refactor (#4874) Signed-off-by: Haytham Abuelfutuh Signed-off-by: Kevin Su Co-authored-by: Haytham Abuelfutuh * bringing ExecutionEnvAssignments from flytekit to propeller ExecutionConfig Signed-off-by: Daniel Rammer * fasttask environment extant working e2e Signed-off-by: Daniel Rammer * using InMemoryEnvironmentBuilder for fasttask Signed-off-by: Daniel Rammer * removed some complexity from ExecutionEnvClient interface Signed-off-by: Daniel Rammer * copied fasttask repo here Signed-off-by: Daniel Rammer * building fasttask plugin into propeller Signed-off-by: Daniel Rammer * moved ExecutionEnvClient Signed-off-by: Daniel Rammer * reverting Makefile skipping flyteconsole change Signed-off-by: Daniel Rammer * docs Signed-off-by: Daniel Rammer * fixing flyteplugins import Signed-off-by: Daniel Rammer * fixed unit tests Signed-off-by: Daniel Rammer * fixed linter Signed-off-by: Daniel Rammer * fixed codespell issues Signed-off-by: Daniel Rammer * codespell attempt 2 Signed-off-by: Daniel Rammer * copying fasttask to single binary Signed-off-by: Daniel Rammer --------- Signed-off-by: Daniel Rammer Signed-off-by: Haytham Abuelfutuh Signed-off-by: Kevin Su Co-authored-by: Kevin Su Co-authored-by: Haytham Abuelfutuh --- .codespellrc | 2 +- Dockerfile | 1 + Dockerfile.flytepropeller | 1 + Dockerfile.sandbox-lite | 1 + fasttask/README.md | 17 + fasttask/buf.gen.yaml | 10 + fasttask/buf.work.yaml | 3 + fasttask/generate_protos.sh | 2 + fasttask/plugin/builder.go | 560 +++++++ fasttask/plugin/config.go | 59 + fasttask/plugin/config_flags.go | 67 + fasttask/plugin/config_flags_test.go | 284 ++++ fasttask/plugin/go.mod | 124 ++ fasttask/plugin/go.sum | 461 ++++++ fasttask/plugin/pb/fasttask.pb.go | 747 +++++++++ fasttask/plugin/pb/fasttask_grpc.pb.go | 141 ++ fasttask/plugin/plugin.go | 317 ++++ fasttask/plugin/service.go | 299 ++++ fasttask/plugin/utils.go | 32 + fasttask/protos/fasttask.proto | 63 + fasttask/worker/Cargo.lock | 1458 +++++++++++++++++ fasttask/worker/Cargo.toml | 28 + fasttask/worker/Dockerfile | 8 + fasttask/worker/src/bridge.rs | 311 ++++ fasttask/worker/src/executor.rs | 98 ++ fasttask/worker/src/main.rs | 99 ++ fasttask/worker/src/pb/fasttask.rs | 118 ++ fasttask/worker/src/pb/fasttask.tonic.rs | 302 ++++ fasttask/worker/src/pb/mod.rs | 1 + fasttask/worker/src/task.rs | 319 ++++ flyteadmin/pkg/manager/impl/shared/iface.go | 2 + flyteadmin/pkg/manager/impl/util/shared.go | 5 + .../interfaces/application_configuration.go | 6 + .../workflowengine/impl/prepare_execution.go | 7 + .../gen/pb-es/flyteidl/admin/execution_pb.ts | 9 + .../pb-es/flyteidl/admin/launch_plan_pb.ts | 9 + .../flyteidl/admin/matchable_resource_pb.ts | 9 + .../pb-es/flyteidl/core/execution_envs_pb.ts | 141 ++ .../gen/pb-go/flyteidl/admin/execution.pb.go | 796 ++++----- .../pb-go/flyteidl/admin/launch_plan.pb.go | 437 ++--- .../flyteidl/admin/matchable_resource.pb.go | 449 ++--- .../pb-go/flyteidl/core/execution_envs.pb.go | 321 ++++ .../flyteidl/artifacts/artifacts.swagger.json | 51 + .../flyteidl/core/execution_envs.swagger.json | 46 + .../flyteidl/service/admin.swagger.json | 67 + flyteidl/gen/pb-js/flyteidl.d.ts | 155 ++ flyteidl/gen/pb-js/flyteidl.js | 423 +++++ .../pb_python/flyteidl/admin/execution_pb2.py | 123 +- .../flyteidl/admin/execution_pb2.pyi | 7 +- .../flyteidl/admin/launch_plan_pb2.py | 55 +- .../flyteidl/admin/launch_plan_pb2.pyi | 7 +- .../flyteidl/admin/matchable_resource_pb2.py | 63 +- .../flyteidl/admin/matchable_resource_pb2.pyi | 7 +- .../flyteidl/core/execution_envs_pb2.py | 30 + .../flyteidl/core/execution_envs_pb2.pyi | 29 + .../flyteidl/core/execution_envs_pb2_grpc.py | 4 + flyteidl/gen/pb_rust/flyteidl.admin.rs | 9 + flyteidl/gen/pb_rust/flyteidl.core.rs | 44 + .../protos/flyteidl/admin/execution.proto | 4 + .../protos/flyteidl/admin/launch_plan.proto | 4 + .../flyteidl/admin/matchable_resource.proto | 4 + .../protos/flyteidl/core/execution_envs.proto | 39 + .../pluginmachinery/core/exec_context.go | 3 + .../pluginmachinery/core/execution_env.go | 14 + .../core/mocks/execution_env_client.go | 91 + .../core/mocks/task_execution_context.go | 34 + .../core/mocks/task_overrides.go | 34 - .../go/tasks/plugins/k8s/ray/config_flags.go | 1 + .../plugins/k8s/ray/config_flags_test.go | 14 + flytepropeller/go.mod | 2 + .../v1alpha1/execution_config.go | 25 + ...ion.multi_images.my_workflow_2_wf_crd.json | 2 +- ...inerization.raw_container.wf_2_wf_crd.json | 2 +- ...e_secrets.my_secret_workflow_2_wf_crd.json | 2 +- ...w.chain_tasks.chain_tasks_wf_2_wf_crd.json | 2 +- ...trol_flow.checkpoint.example_2_wf_crd.json | 2 +- ...l_flow.conditions.multiplier_2_wf_crd.json | 2 +- ...flow.conditions.multiplier_2_2_wf_crd.json | 2 +- ...flow.conditions.multiplier_3_2_wf_crd.json | 2 +- ....conditions.basic_boolean_wf_2_wf_crd.json | 2 +- ...low.conditions.bool_input_wf_2_wf_crd.json | 2 +- ...conditions.nested_conditions_2_wf_crd.json | 2 +- ...w.conditions.consume_outputs_2_wf_crd.json | 2 +- ...ore.control_flow.dynamics.wf_2_wf_crd.json | 2 +- ...low.map_task.my_map_workflow_2_wf_crd.json | 2 +- ...l_flow.merge_sort.merge_sort_2_wf_crd.json | 2 +- ...l_flow.subworkflows.my_subwf_2_wf_crd.json | 2 +- ...ow.subworkflows.ext_workflow_2_wf_crd.json | 2 +- ...stom_task_plugin.my_workflow_2_wf_crd.json | 2 +- ...extend_flyte.custom_types.wf_2_wf_crd.json | 2 +- ..._basics.basic_workflow.my_wf_2_wf_crd.json | 2 +- ...e_basics.decorating_tasks.wf_2_wf_crd.json | 2 +- ...sics.decorating_workflows.wf_2_wf_crd.json | 2 +- ...ed_workflow.sphinx_docstring_2_wf_crd.json | 2 +- ...ted_workflow.numpy_docstring_2_wf_crd.json | 2 +- ...ed_workflow.google_docstring_2_wf_crd.json | 2 +- ...ics.files.normalize_csv_file_2_wf_crd.json | 2 +- ...load_and_normalize_csv_files_2_wf_crd.json | 2 +- ...yte_basics.hello_world.my_wf_2_wf_crd.json | 2 +- ....imperative.workflow.example_2_wf_crd.json | 2 +- ...0_core.flyte_basics.lp.my_wf_2_wf_crd.json | 2 +- ...ore.flyte_basics.lp.go_greet_2_wf_crd.json | 2 +- ...e_basics.named_outputs.my_wf_2_wf_crd.json | 2 +- ...e.flyte_basics.shell_task.wf_2_wf_crd.json | 2 +- ...sk_cache.cached_dataframe_wf_2_wf_crd.json | 2 +- ..._schedules.date_formatter_wf_2_wf_crd.json | 2 +- ...ows.lp_schedules.positive_wf_2_wf_crd.json | 2 +- ...ype_system.custom_objects.wf_2_wf_crd.json | 2 +- ...re.type_system.enums.enum_wf_2_wf_crd.json | 2 +- ..._system.flyte_pickle.welcome_2_wf_crd.json | 2 +- ...ore.type_system.schema.df_wf_2_wf_crd.json | 2 +- ...aset.pandas_compatibility_wf_2_wf_crd.json | 2 +- ...aset.schema_compatibility_wf_2_wf_crd.json | 2 +- ....type_system.typed_schema.wf_2_wf_crd.json | 2 +- flytepropeller/pkg/controller/controller.go | 10 +- .../pkg/controller/execution_env.go | 44 + .../pkg/controller/nodes/array/handler.go | 2 +- .../controller/nodes/array/handler_test.go | 10 +- .../nodes/dynamic/dynamic_workflow_test.go | 2 +- .../pkg/controller/nodes/executor.go | 5 +- .../pkg/controller/nodes/executor_test.go | 64 +- .../mocks/node_execution_context.go | 36 + .../nodes/interfaces/node_exec_context.go | 4 + .../pkg/controller/nodes/node_exec_context.go | 36 +- .../nodes/node_exec_context_test.go | 2 +- .../pkg/controller/workflow/executor.go | 41 +- .../pkg/controller/workflow/executor_test.go | 61 +- flytepropeller/plugins/loader.go | 3 + go.mod | 2 + 129 files changed, 8819 insertions(+), 1102 deletions(-) create mode 100644 fasttask/README.md create mode 100644 fasttask/buf.gen.yaml create mode 100644 fasttask/buf.work.yaml create mode 100755 fasttask/generate_protos.sh create mode 100644 fasttask/plugin/builder.go create mode 100644 fasttask/plugin/config.go create mode 100755 fasttask/plugin/config_flags.go create mode 100755 fasttask/plugin/config_flags_test.go create mode 100644 fasttask/plugin/go.mod create mode 100644 fasttask/plugin/go.sum create mode 100644 fasttask/plugin/pb/fasttask.pb.go create mode 100644 fasttask/plugin/pb/fasttask_grpc.pb.go create mode 100644 fasttask/plugin/plugin.go create mode 100644 fasttask/plugin/service.go create mode 100644 fasttask/plugin/utils.go create mode 100644 fasttask/protos/fasttask.proto create mode 100644 fasttask/worker/Cargo.lock create mode 100644 fasttask/worker/Cargo.toml create mode 100644 fasttask/worker/Dockerfile create mode 100644 fasttask/worker/src/bridge.rs create mode 100644 fasttask/worker/src/executor.rs create mode 100644 fasttask/worker/src/main.rs create mode 100644 fasttask/worker/src/pb/fasttask.rs create mode 100644 fasttask/worker/src/pb/fasttask.tonic.rs create mode 100644 fasttask/worker/src/pb/mod.rs create mode 100644 fasttask/worker/src/task.rs create mode 100644 flyteidl/gen/pb-es/flyteidl/core/execution_envs_pb.ts create mode 100644 flyteidl/gen/pb-go/flyteidl/core/execution_envs.pb.go create mode 100644 flyteidl/gen/pb-go/gateway/flyteidl/core/execution_envs.swagger.json create mode 100644 flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.py create mode 100644 flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.pyi create mode 100644 flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2_grpc.py create mode 100644 flyteidl/protos/flyteidl/core/execution_envs.proto create mode 100644 flyteplugins/go/tasks/pluginmachinery/core/execution_env.go create mode 100644 flyteplugins/go/tasks/pluginmachinery/core/mocks/execution_env_client.go create mode 100644 flytepropeller/pkg/controller/execution_env.go diff --git a/.codespellrc b/.codespellrc index 0f0abe525b..0606280794 100644 --- a/.codespellrc +++ b/.codespellrc @@ -3,4 +3,4 @@ skip = .git,*.pdf,*.svg,go.sum,go.mod,*requirements.txt,gen # some strings with unicodes, constructs like [o]utput ignore-regex = ".*\\0[0-9][0-9].*"|json:"[^"]*"|\b[a-z]*\[[a-z]\][a-z]*\b # some ad-hoc variable names etc -ignore-words-list = te,nd,querys,ser +ignore-words-list = crate,te,nd,querys,ser diff --git a/Dockerfile b/Dockerfile index e25463157f..1eeb4457f2 100644 --- a/Dockerfile +++ b/Dockerfile @@ -12,6 +12,7 @@ WORKDIR /flyteorg/build COPY cacheservice cacheservice COPY datacatalog datacatalog +COPY fasttask fasttask COPY flyteadmin flyteadmin COPY flytecopilot flytecopilot COPY flyteidl flyteidl diff --git a/Dockerfile.flytepropeller b/Dockerfile.flytepropeller index d524fbcf4a..5733c7ff65 100644 --- a/Dockerfile.flytepropeller +++ b/Dockerfile.flytepropeller @@ -15,6 +15,7 @@ WORKDIR /go/src/github.com/flyteorg/flytepropeller COPY cacheservice ../cacheservice COPY datacatalog ../datacatalog +COPY fasttask ../fasttask COPY flyteadmin ../flyteadmin COPY flytecopilot ../flytecopilot COPY flyteidl ../flyteidl diff --git a/Dockerfile.sandbox-lite b/Dockerfile.sandbox-lite index 92414120e2..8f8a5c3387 100644 --- a/Dockerfile.sandbox-lite +++ b/Dockerfile.sandbox-lite @@ -17,6 +17,7 @@ WORKDIR /app/flyte COPY cacheservice cacheservice COPY datacatalog datacatalog +COPY fasttask fastask COPY flyteadmin flyteadmin COPY flytecopilot flytecopilot COPY flyteidl flyteidl diff --git a/fasttask/README.md b/fasttask/README.md new file mode 100644 index 0000000000..f5628e1905 --- /dev/null +++ b/fasttask/README.md @@ -0,0 +1,17 @@ +# fast task +FastTask is a Flyte plugin to execute tasks quickly using a persistent, external worker; reducing the overhead of cold-starting a k8s Pod as an execution environment. This enables Flyte to execute short tasks in parallel, without the overhead of starting a new Pod for each task; instead sharing a pool of workers across multiple tasks and workflow executions. + +## development setup + # get flytekit in terminal + source ~/Development/flytekit/.venv-union/bin/activate + + # create temporary python dir for fast registration + mkdir /tmp/fasttask-test + export PYTHONPATH=$PYTHONPATH:/tmp/fasttask-test + + # allow flytekit to connect to minio cluster + export FLYTE_AWS_ACCESS_KEY_ID=minio + export FLYTE_AWS_SECRET_ACCESS_KEY=miniostorage + export FLYTE_AWS_ENDPOINT=http://localhost:30084 +## build fast task worker image + docker build -t hamersaw/fasttask: -f Dockerfile . diff --git a/fasttask/buf.gen.yaml b/fasttask/buf.gen.yaml new file mode 100644 index 0000000000..0d377e528f --- /dev/null +++ b/fasttask/buf.gen.yaml @@ -0,0 +1,10 @@ +version: v1 +plugins: + - plugin: buf.build/protocolbuffers/go:v1.28.1 + out: plugin + - plugin: buf.build/grpc/go:v1.3.0 + out: plugin + - plugin: buf.build/community/neoeinstein-prost:v0.2.3 + out: worker/src/pb + - plugin: buf.build/community/neoeinstein-tonic:v0.3.0 + out: worker/src/pb diff --git a/fasttask/buf.work.yaml b/fasttask/buf.work.yaml new file mode 100644 index 0000000000..4bddfe6792 --- /dev/null +++ b/fasttask/buf.work.yaml @@ -0,0 +1,3 @@ +version: v1 +directories: + - protos diff --git a/fasttask/generate_protos.sh b/fasttask/generate_protos.sh new file mode 100755 index 0000000000..485be16602 --- /dev/null +++ b/fasttask/generate_protos.sh @@ -0,0 +1,2 @@ +#!/bin/bash +docker run --rm -u $(id -u):$(id -g) -e "BUF_CACHE_DIR=/tmp/cache" --volume "$(pwd):/workspace" --workdir /workspace bufbuild/buf generate diff --git a/fasttask/plugin/builder.go b/fasttask/plugin/builder.go new file mode 100644 index 0000000000..7bc3613cd6 --- /dev/null +++ b/fasttask/plugin/builder.go @@ -0,0 +1,560 @@ +package plugin + +import ( + "context" + "encoding/hex" + "encoding/json" + "fmt" + "math/rand" + "strconv" + "sync" + "time" + + "k8s.io/api/core/v1" + k8serrors "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" + "k8s.io/apimachinery/pkg/util/wait" + "sigs.k8s.io/controller-runtime/pkg/client" + _struct "github.com/golang/protobuf/ptypes/struct" + + flyteerrors "github.com/flyteorg/flyte/flyteplugins/go/tasks/errors" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/utils" + "github.com/flyteorg/flyte/flytestdlib/logger" + "github.com/unionai/flyte/fasttask/plugin/pb" +) + +type state int32 + +const ( + HEALTHY state = iota + ORPHANED + REPAIRING + TOMBSTONED +) + +const ( + EXECUTION_ENV_ID = "execution-env-id" + EXECUTION_ENV_TYPE = "execution-env-type" + TTL_SECONDS = "ttl-seconds" +) + +// environment represents a managed fast task environment, including it's definition and current +// state +type environment struct { + lastAccessedAt time.Time + extant *_struct.Struct + replicas []string + spec *pb.FastTaskEnvironmentSpec + state state +} + +// InMemoryEnvBuilder is an in-memory implementation of the ExecutionEnvBuilder interface. It is +// used to manage the lifecycle of fast task environments. +type InMemoryEnvBuilder struct { + environments map[string]*environment + kubeClient core.KubeClient + lock sync.Mutex + randSource *rand.Rand +} + +// Get retrieves the environment with the given execution environment ID. If the environment does +// not exist or has been tombstoned, nil is returned. +func (i *InMemoryEnvBuilder) Get(ctx context.Context, executionEnvID string) *_struct.Struct { + if environment := i.environments[executionEnvID]; environment != nil { + i.lock.Lock() + defer i.lock.Unlock() + + if environment.state != TOMBSTONED { + environment.lastAccessedAt = time.Now() + return environment.extant + } + } + return nil +} + +// Create creates a new fast task environment with the given execution environment ID and +// specification. If the environment already exists, the existing environment is returned. +func (i *InMemoryEnvBuilder) Create(ctx context.Context, executionEnvID string, spec *_struct.Struct) (*_struct.Struct, error) { + // unmarshall and validate FastTaskEnvironmentSpec + fastTaskEnvironmentSpec := &pb.FastTaskEnvironmentSpec{} + if err := utils.UnmarshalStruct(spec, fastTaskEnvironmentSpec); err != nil { + return nil, err + } + + if err := isValidEnvironmentSpec(fastTaskEnvironmentSpec); err != nil { + return nil, flyteerrors.Errorf(flyteerrors.BadTaskSpecification, + "detected invalid FastTaskEnvironmentSpec [%v], Err: [%v]", fastTaskEnvironmentSpec.PodTemplateSpec, err) + } + + logger.Debug(ctx, "creating environment '%s'", executionEnvID) + + // build fastTaskEnvironment extant + fastTaskEnvironment := &pb.FastTaskEnvironment{ + QueueId: executionEnvID, + } + environmentStruct := &_struct.Struct{} + if err := utils.MarshalStruct(fastTaskEnvironment, environmentStruct); err != nil { + return nil, fmt.Errorf("unable to marshal ExecutionEnv [%v], Err: [%v]", fastTaskEnvironment, err.Error()) + } + + // create environment + i.lock.Lock() + + env, exists := i.environments[executionEnvID] + if exists && env.state != ORPHANED { + i.lock.Unlock() + + // if exists we created from another task in race condition between `Get` and `Create` + return env.extant, nil + } + + var replicas []string + if env != nil { + // if environment already exists then copy existing replicas + replicas = env.replicas + } else { + replicas = make([]string, 0) + } + + env = &environment{ + lastAccessedAt: time.Now(), + extant: environmentStruct, + replicas: replicas, + spec: fastTaskEnvironmentSpec, + state: HEALTHY, + } + + podNames := make([]string, 0) + for replica := len(env.replicas); replica < int(fastTaskEnvironmentSpec.ReplicaCount); replica++ { + nonceBytes := make([]byte, (GetConfig().NonceLength+1)/2) + if _, err := i.randSource.Read(nonceBytes); err != nil { + return nil, err + } + + podName := fmt.Sprintf("%s-%s", executionEnvID, hex.EncodeToString(nonceBytes)[:GetConfig().NonceLength]) + env.replicas = append(env.replicas, podName) + podNames = append(podNames, podName) + } + + i.environments[executionEnvID] = env + i.lock.Unlock() + + // create replicas + for _, podName := range podNames { + logger.Debugf(ctx, "creating pod '%s' for environment '%s'", podName, executionEnvID) + if err := i.createPod(ctx, fastTaskEnvironmentSpec, executionEnvID, podName); err != nil { + logger.Warnf(ctx, "failed to create pod '%s' for environment '%s' [%v]", podName, executionEnvID, err) + } + } + + logger.Infof(ctx, "created environment '%s'", executionEnvID) + return env.extant, nil +} + +// createPod creates a new pod for the given execution environment ID and pod name. The pod is +// created using the given FastTaskEnvironmentSpec. +func (i *InMemoryEnvBuilder) createPod(ctx context.Context, fastTaskEnvironmentSpec *pb.FastTaskEnvironmentSpec, executionEnvID, podName string) error { + podTemplateSpec := &v1.PodTemplateSpec{} + if err := json.Unmarshal(fastTaskEnvironmentSpec.PodTemplateSpec, podTemplateSpec); err != nil { + return flyteerrors.Errorf(flyteerrors.BadTaskSpecification, + "unable to unmarshal PodTemplateSpec [%v], Err: [%v]", fastTaskEnvironmentSpec.PodTemplateSpec, err.Error()) + } + + podSpec := &podTemplateSpec.Spec + objectMeta := &podTemplateSpec.ObjectMeta + + // identify the primary container + primaryContainerIndex := -1 + if len(fastTaskEnvironmentSpec.PrimaryContainerName) > 0 { + for index, container := range podSpec.Containers { + if container.Name == fastTaskEnvironmentSpec.PrimaryContainerName { + primaryContainerIndex = index + break + } + } + } else if len(podSpec.Containers) == 1 { + primaryContainerIndex = 0 + } + + if primaryContainerIndex == -1 { + return flyteerrors.Errorf(flyteerrors.BadTaskSpecification, + "unable to identify primary container in the environment spec [%v]", podSpec) + } + + // add execution environment labels and annotations + objectMeta.Name = podName + if objectMeta.Labels == nil { + objectMeta.Labels = make(map[string]string, 0) + } + objectMeta.Labels[EXECUTION_ENV_TYPE] = fastTaskType + objectMeta.Labels[EXECUTION_ENV_ID] = executionEnvID + if objectMeta.Annotations == nil { + objectMeta.Annotations = make(map[string]string, 0) + } + objectMeta.Annotations[TTL_SECONDS] = fmt.Sprintf("%d", fastTaskEnvironmentSpec.GetTtlSeconds()) + + // create new volume 'workdir' + podSpec.Volumes = append(podSpec.Volumes, v1.Volume{ + Name: "workdir", + VolumeSource: v1.VolumeSource{ + EmptyDir: &v1.EmptyDirVolumeSource{}, + }, + }) + + // add init container to copy worker binary to volume + podSpec.InitContainers = append(podSpec.InitContainers, v1.Container{ + Name: "worker", + Image: GetConfig().Image, + ImagePullPolicy: v1.PullIfNotPresent, + Command: []string{"cp", "/usr/local/bin/worker", "/tmp/worker"}, + Resources: v1.ResourceRequirements{ + Limits: v1.ResourceList{ + v1.ResourceCPU: GetConfig().InitContainerCPU, + v1.ResourceMemory: GetConfig().InitContainerMemory, + }, + Requests: v1.ResourceList{ + v1.ResourceCPU: GetConfig().InitContainerCPU, + v1.ResourceMemory: GetConfig().InitContainerMemory, + }, + }, + VolumeMounts: []v1.VolumeMount{ + { + Name: "workdir", + MountPath: "/tmp", + }, + }, + }) + + // update primary container arguments and volume mounts + container := &podSpec.Containers[primaryContainerIndex] + container.Args = []string{ + "/tmp/worker", + "bridge", + "--queue-id", + executionEnvID, + "--fasttask-url", + GetConfig().CallbackURI, + } + + if fastTaskEnvironmentSpec.BacklogLength > 0 { + container.Args = append(container.Args, "--backlog-length", fmt.Sprintf("%d", fastTaskEnvironmentSpec.BacklogLength)) + } + if fastTaskEnvironmentSpec.Parallelism > 0 { + container.Args = append(container.Args, "--parallelism", fmt.Sprintf("%d", fastTaskEnvironmentSpec.Parallelism)) + } + + container.VolumeMounts = append(container.VolumeMounts, v1.VolumeMount{ + Name: "workdir", + MountPath: "/tmp", + }) + + // use kubeclient to create worker + return i.kubeClient.GetClient().Create(ctx, &v1.Pod{ + ObjectMeta: *objectMeta, + Spec: *podSpec, + }) +} + +// deletePod deletes the pod with the given name and namespace. +func (i *InMemoryEnvBuilder) deletePod(ctx context.Context, name types.NamespacedName) error { + objectMeta := metav1.ObjectMeta{ + Name: name.Name, + Namespace: name.Namespace, + } + + return i.kubeClient.GetClient().Delete(ctx, &v1.Pod{ + ObjectMeta: objectMeta, + }, client.GracePeriodSeconds(0)) +} + +// Start starts the environment garbage collector, orphan detection, and environment repair +// processes. +func (i *InMemoryEnvBuilder) Start(ctx context.Context) error { + // detect orphaned environments using k8s client + if err := i.detectOrphanedEnvironments(ctx, i.kubeClient.GetClient()); err != nil { + return err + } + + // start environment garbage collector + go func() { + wait.UntilWithContext(ctx, + func(ctx context.Context) { + if err := i.gcEnvironments(ctx); err != nil { + logger.Warnf(ctx, "failed to gc environment(s) [%v]", err) + } + }, + GetConfig().EnvGCInterval.Duration, + ) + }() + + // start environment repair + go func() { + wait.UntilWithContext(ctx, + func(ctx context.Context) { + if err := i.repairEnvironments(ctx); err != nil { + logger.Warnf(ctx, "failed to repair environment(s) [%v]", err) + } + }, + GetConfig().EnvRepairInterval.Duration, + ) + }() + + // start orphan detection + go func() { + wait.UntilWithContext(ctx, + func(ctx context.Context) { + if err := i.detectOrphanedEnvironments(ctx, i.kubeClient.GetCache()); err != nil { + logger.Warnf(ctx, "failed to detect orphaned environment(s) [%v]", err) + } + }, + GetConfig().EnvDetectOrphanInterval.Duration, + ) + }() + + return nil +} + +// gcEnvironments garbage collects environments that have expired based on their termination +// criteria. +func (i *InMemoryEnvBuilder) gcEnvironments(ctx context.Context) error { + // identify environments that have expired + now_seconds := time.Now().Unix() + environmentReplicas := make(map[string][]types.NamespacedName, 0) + + i.lock.Lock() + for environmentID, environment := range i.environments { + if environment.state == REPAIRING { + continue + } + + // if the environment has a ttlSeconds termination criteria then check if it has expired + if ttlCriteria, ok := environment.spec.GetTerminationCriteria().(*pb.FastTaskEnvironmentSpec_TtlSeconds); ok { + if environment.state == TOMBSTONED || now_seconds - environment.lastAccessedAt.Unix() >= int64(ttlCriteria.TtlSeconds) { + environment.state = TOMBSTONED + + podTemplateSpec := &v1.PodTemplateSpec{} + if err := json.Unmarshal(environment.spec.PodTemplateSpec, podTemplateSpec); err != nil { + return flyteerrors.Errorf(flyteerrors.BadTaskSpecification, + "unable to unmarshal PodTemplateSpec [%v], Err: [%v]", environment.spec.PodTemplateSpec, err.Error()) + } + + podNames := make([]types.NamespacedName, 0) + for _, podName := range environment.replicas { + podNames = append(podNames, + types.NamespacedName{ + Name: podName, + Namespace: podTemplateSpec.Namespace, + }) + } + + logger.Infof(ctx, "tombstoning environment '%s'", environmentID) + environmentReplicas[environmentID] = podNames + } + } + } + i.lock.Unlock() + + // delete environments + deletedEnvironments := make([]string, 0) + for environmentID, podNames := range environmentReplicas { + deleted := true + for _, podName := range podNames { + logger.Debugf(ctx, "deleting pod '%s' for environment '%s'", podName, environmentID) + err := i.deletePod(ctx, podName) + if err != nil && !k8serrors.IsNotFound(err) { + logger.Warnf(ctx, "failed to gc pod '%s' for environment '%s' [%v]", podName, environmentID, err) + deleted = false + } + } + + if deleted { + deletedEnvironments = append(deletedEnvironments, environmentID) + } + } + + // remove deleted environments + i.lock.Lock() + for _, environmentID := range deletedEnvironments { + logger.Infof(ctx, "garbage collected environment '%s'", environmentID) + delete(i.environments, environmentID) + } + i.lock.Unlock() + + return nil +} + +// repairEnvironments repairs environments that have been externally modified (ie. pod deletion). +func (i *InMemoryEnvBuilder) repairEnvironments(ctx context.Context) error { + environmentSpecs := make(map[string]pb.FastTaskEnvironmentSpec, 0) + environmentReplicas := make(map[string][]string, 0) + + // identify environments in need of repair + i.lock.Lock() + pod := &v1.Pod{} + for environmentID, environment := range i.environments { + // check if environment is repairable (ie. HEALTHY or REPAIRING state) + if environment.state != HEALTHY && environment.state != REPAIRING { + continue + } + + podTemplateSpec := &v1.PodTemplateSpec{} + if err := json.Unmarshal(environment.spec.PodTemplateSpec, podTemplateSpec); err != nil { + return flyteerrors.Errorf(flyteerrors.BadTaskSpecification, + "unable to unmarshal PodTemplateSpec [%v], Err: [%v]", environment.spec.PodTemplateSpec, err.Error()) + } + + podNames := make([]string, 0) + for index, podName := range environment.replicas { + err := i.kubeClient.GetCache().Get(ctx, types.NamespacedName{ + Name: podName, + Namespace: podTemplateSpec.Namespace, + }, pod) + + if k8serrors.IsNotFound(err) || k8serrors.IsGone(err) { + nonceBytes := make([]byte, (GetConfig().NonceLength+1)/2) + if _, err := i.randSource.Read(nonceBytes); err != nil { + return err + } + + newPodName := fmt.Sprintf("%s-%s", environmentID, hex.EncodeToString(nonceBytes)[:GetConfig().NonceLength]) + environment.replicas[index] = newPodName + podNames = append(podNames, newPodName) + } + } + + if len(podNames) > 0 { + logger.Infof(ctx, "repairing environment '%s'", environmentID) + environment.state = REPAIRING + environmentSpecs[environmentID] = *environment.spec + environmentReplicas[environmentID] = podNames + } + } + i.lock.Unlock() + + // attempt to repair replicas + for environmentID, environmentSpec := range environmentSpecs { + for _, podName := range environmentReplicas[environmentID] { + logger.Debugf(ctx, "creating pod '%s' for environment '%s'", podName, environmentID) + if err := i.createPod(ctx, &environmentSpec, environmentID, podName); err != nil { + logger.Warnf(ctx, "failed to create pod '%s' for environment '%s' [%v]", podName, environmentID, err) + } + } + } + + // transition repaired environments to HEALTHY + i.lock.Lock() + for environmentID := range environmentSpecs { + environment, exists := i.environments[environmentID] + if !exists { + // this should be unreachable as repair / gc operations use the same lock to ensure + // concurrent operations do not interfere with each other + logger.Warnf(ctx, "environment '%s' was deleted during repair operation", environmentID) + continue + } + + logger.Infof(ctx, "repaired environment '%s'", environmentID) + environment.state = HEALTHY + } + i.lock.Unlock() + + return nil +} + +// detectOrphanedEnvironments detects orphaned environments by identifying pods with the fast task +// execution type label. +func (i *InMemoryEnvBuilder) detectOrphanedEnvironments(ctx context.Context, k8sReader client.Reader) error { + // retrieve all pods with fast task execution type label + var matchingLabelsOption client.MatchingLabels + matchingLabelsOption = make(map[string]string) + matchingLabelsOption[EXECUTION_ENV_TYPE] = fastTaskType + + podList := &v1.PodList{} + if err := k8sReader.List(ctx, podList, matchingLabelsOption); err != nil { + return err + } + + // detect orphaned environments + i.lock.Lock() + defer i.lock.Unlock() + + orphanedEnvironments := make(map[string]*environment, 0) + for _, pod := range podList.Items { + // if environment exists we do not need to process + environmentID, labelExists := pod.Labels[EXECUTION_ENV_ID] + if !labelExists { + continue + } + + _, environmentExists := i.environments[environmentID] + if environmentExists { + continue + } + + // create or add pod to orphaned environment + orphanedEnvironment, exists := orphanedEnvironments[environmentID] + if !exists { + // serialize podTemplateSpec with Namespace so we can use it to delete TOMBSTONED environments + podTemplateSpec := &v1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Namespace: pod.Namespace, + }, + } + + podTemplateSpecBytes, err := json.Marshal(podTemplateSpec) + if err != nil { + logger.Warnf(ctx, "failed to marshal PodTemplateSpec [%v]", err) + continue + } + + // parse ttl seconds from annotations + ttlSeconds := 0 + if ttlSecondsStr, exists := pod.Annotations[TTL_SECONDS]; exists { + ttlSeconds, err = strconv.Atoi(ttlSecondsStr) + if err != nil { + // this should be unreachable because we are serializing the integer which + // sets this annotation. if parsing errors then we leave ttlSeconds as 0 to + // ensure the orphaned environment is garbage collected immediately + ttlSeconds = 0 + logger.Warnf(ctx, "failed to parse TTL_SECONDS [%s] for pod '%s' [%v]", ttlSecondsStr, pod.Name, err) + } + } + + // create orphaned environment + orphanedEnvironment = &environment{ + lastAccessedAt: time.Now(), + extant: nil, + replicas: make([]string, 0), + spec: &pb.FastTaskEnvironmentSpec{ + PodTemplateSpec: podTemplateSpecBytes, + TerminationCriteria: &pb.FastTaskEnvironmentSpec_TtlSeconds{ + TtlSeconds: int32(ttlSeconds), + }, + }, + state: ORPHANED, + } + + orphanedEnvironments[environmentID] = orphanedEnvironment + } + + orphanedEnvironment.replicas = append(orphanedEnvironment.replicas, pod.Name) + } + + // copy orphaned environments to env builder + for environmentID, orphanedEnvironment := range orphanedEnvironments { + logger.Infof(ctx, "detected orphaned environment '%s'", environmentID) + i.environments[environmentID] = orphanedEnvironment + } + + return nil +} + +// NewEnvironmentBuilder creates a new InMemoryEnvBuilder with the given kube client. +func NewEnvironmentBuilder(kubeClient core.KubeClient) *InMemoryEnvBuilder { + return &InMemoryEnvBuilder{ + environments: make(map[string]*environment), + kubeClient: kubeClient, + randSource: rand.New(rand.NewSource(time.Now().UnixNano())), + } +} diff --git a/fasttask/plugin/config.go b/fasttask/plugin/config.go new file mode 100644 index 0000000000..21d356447d --- /dev/null +++ b/fasttask/plugin/config.go @@ -0,0 +1,59 @@ +package plugin + +import ( + "time" + + "k8s.io/apimachinery/pkg/api/resource" + + pluginsConfig "github.com/flyteorg/flyte/flyteplugins/go/tasks/config" + "github.com/flyteorg/flyte/flytestdlib/config" +) + +//go:generate pflags Config --default-var=defaultConfig + +var ( + defaultCPU = resource.MustParse("500m") + defaultMemory = resource.MustParse("128Mi") + defaultConfig = &Config{ + CallbackURI: "http://host.k3d.internal:15605", + Endpoint: "0.0.0.0:15605", + EnvDetectOrphanInterval: config.Duration{Duration: time.Second * 60}, + EnvGCInterval: config.Duration{Duration: time.Second * 5}, + EnvRepairInterval: config.Duration{Duration: time.Second * 10}, + GracePeriodStatusNotFound: config.Duration{Duration: time.Second * 15}, + GracePeriodWorkersUnavailable: config.Duration{Duration: time.Second * 30}, + HeartbeatBufferSize: 512, + Image: "flyteorg/fasttask:latest", + InitContainerCPU: defaultCPU, + InitContainerMemory: defaultMemory, + NonceLength: 12, + TaskStatusBufferSize: 512, + } + + configSection = pluginsConfig.MustRegisterSubSection("fasttask", defaultConfig) +) + +type Config struct { + CallbackURI string `json:"callback-uri" pflag:",Fasttask gRPC service URI that fasttask workers will connect to."` + Endpoint string `json:"endpoint" pflag:",Fasttask gRPC service endpoint."` + EnvDetectOrphanInterval config.Duration `json:"env-detect-orphan-interval" pflag:",Frequency that orphaned environments detection is performed."` + EnvGCInterval config.Duration `json:"env-gc-interval" pflag:",Frequency that environments are GCed in case of TTL expirations."` + EnvRepairInterval config.Duration `json:"env-repair-interval" pflag:",Frequency that environments are repaired in case of external modifications (ex. pod deletion)."` + GracePeriodStatusNotFound config.Duration `json:"grace-period-status-not-found" pflag:",The grace period for a task status to be reported before the task is considered failed."` + GracePeriodWorkersUnavailable config.Duration `json:"grace-period-workers-unavailable" pflag:",The grace period for a worker to become available before the task is considered failed."` + HeartbeatBufferSize int `json:"heartbeat-buffer-size" pflag:",The size of the heartbeat buffer for each worker."` + Image string `json:"image" pflag:",Fasttask image to wrap the task execution with."` + InitContainerCPU resource.Quantity `json:"init-container-cpu" pflag:",The default cpu request / limit for the init container used to inject the fasttask worker binary."` + InitContainerMemory resource.Quantity `json:"init-container-memory" pflag:",The default memory request / limit for the init container used to inject the fasttask worker binary."` + NonceLength int `json:"nonce-length" pflag:",The length of the nonce value to uniquely link a fasttask replica to the environment instance, ensuring fast turnover of environments regardless of cache freshness."` + TaskStatusBufferSize int `json:"task-status-buffer-size" pflag:",The size of the task status buffer for each task."` +} + +func GetConfig() *Config { + return configSection.GetConfig().(*Config) +} + +// This method should be used for unit testing only +func setConfig(cfg *Config) error { + return configSection.SetConfig(cfg) +} diff --git a/fasttask/plugin/config_flags.go b/fasttask/plugin/config_flags.go new file mode 100755 index 0000000000..de80d49152 --- /dev/null +++ b/fasttask/plugin/config_flags.go @@ -0,0 +1,67 @@ +// Code generated by go generate; DO NOT EDIT. +// This file was generated by robots. + +package plugin + +import ( + "encoding/json" + "reflect" + + "fmt" + + "github.com/spf13/pflag" +) + +// If v is a pointer, it will get its element value or the zero value of the element type. +// If v is not a pointer, it will return it as is. +func (Config) elemValueOrNil(v interface{}) interface{} { + if t := reflect.TypeOf(v); t.Kind() == reflect.Ptr { + if reflect.ValueOf(v).IsNil() { + return reflect.Zero(t.Elem()).Interface() + } else { + return reflect.ValueOf(v).Interface() + } + } else if v == nil { + return reflect.Zero(t).Interface() + } + + return v +} + +func (Config) mustJsonMarshal(v interface{}) string { + raw, err := json.Marshal(v) + if err != nil { + panic(err) + } + + return string(raw) +} + +func (Config) mustMarshalJSON(v json.Marshaler) string { + raw, err := v.MarshalJSON() + if err != nil { + panic(err) + } + + return string(raw) +} + +// GetPFlagSet will return strongly types pflags for all fields in Config and its nested types. The format of the +// flags is json-name.json-sub-name... etc. +func (cfg Config) GetPFlagSet(prefix string) *pflag.FlagSet { + cmdFlags := pflag.NewFlagSet("Config", pflag.ExitOnError) + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "callback-uri"), defaultConfig.CallbackURI, "Fasttask gRPC service URI that fasttask workers will connect to.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "endpoint"), defaultConfig.Endpoint, "Fasttask gRPC service endpoint.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "env-detect-orphan-interval"), defaultConfig.EnvDetectOrphanInterval.String(), "Frequency that orphaned environments detection is performed.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "env-gc-interval"), defaultConfig.EnvGCInterval.String(), "Frequency that environments are GCed in case of TTL expirations.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "env-repair-interval"), defaultConfig.EnvRepairInterval.String(), "Frequency that environments are repaired in case of external modifications (ex. pod deletion).") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "grace-period-status-not-found"), defaultConfig.GracePeriodStatusNotFound.String(), "The grace period for a task status to be reported before the task is considered failed.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "grace-period-workers-unavailable"), defaultConfig.GracePeriodWorkersUnavailable.String(), "The grace period for a worker to become available before the task is considered failed.") + cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "heartbeat-buffer-size"), defaultConfig.HeartbeatBufferSize, "The size of the heartbeat buffer for each worker.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "image"), defaultConfig.Image, "Fasttask image to wrap the task execution with.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "init-container-cpu"), defaultConfig.InitContainerCPU.String(), "The default cpu request / limit for the init container used to inject the fasttask worker binary.") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "init-container-memory"), defaultConfig.InitContainerMemory.String(), "The default memory request / limit for the init container used to inject the fasttask worker binary.") + cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "nonce-length"), defaultConfig.NonceLength, "The length of the nonce value to uniquely link a fasttask replica to the environment instance, ensuring fast turnover of environments regardless of cache freshness.") + cmdFlags.Int(fmt.Sprintf("%v%v", prefix, "task-status-buffer-size"), defaultConfig.TaskStatusBufferSize, "The size of the task status buffer for each task.") + return cmdFlags +} diff --git a/fasttask/plugin/config_flags_test.go b/fasttask/plugin/config_flags_test.go new file mode 100755 index 0000000000..63ff1c9cca --- /dev/null +++ b/fasttask/plugin/config_flags_test.go @@ -0,0 +1,284 @@ +// Code generated by go generate; DO NOT EDIT. +// This file was generated by robots. + +package plugin + +import ( + "encoding/json" + "fmt" + "reflect" + "strings" + "testing" + + "github.com/mitchellh/mapstructure" + "github.com/stretchr/testify/assert" +) + +var dereferencableKindsConfig = map[reflect.Kind]struct{}{ + reflect.Array: {}, reflect.Chan: {}, reflect.Map: {}, reflect.Ptr: {}, reflect.Slice: {}, +} + +// Checks if t is a kind that can be dereferenced to get its underlying type. +func canGetElementConfig(t reflect.Kind) bool { + _, exists := dereferencableKindsConfig[t] + return exists +} + +// This decoder hook tests types for json unmarshaling capability. If implemented, it uses json unmarshal to build the +// object. Otherwise, it'll just pass on the original data. +func jsonUnmarshalerHookConfig(_, to reflect.Type, data interface{}) (interface{}, error) { + unmarshalerType := reflect.TypeOf((*json.Unmarshaler)(nil)).Elem() + if to.Implements(unmarshalerType) || reflect.PtrTo(to).Implements(unmarshalerType) || + (canGetElementConfig(to.Kind()) && to.Elem().Implements(unmarshalerType)) { + + raw, err := json.Marshal(data) + if err != nil { + fmt.Printf("Failed to marshal Data: %v. Error: %v. Skipping jsonUnmarshalHook", data, err) + return data, nil + } + + res := reflect.New(to).Interface() + err = json.Unmarshal(raw, &res) + if err != nil { + fmt.Printf("Failed to umarshal Data: %v. Error: %v. Skipping jsonUnmarshalHook", data, err) + return data, nil + } + + return res, nil + } + + return data, nil +} + +func decode_Config(input, result interface{}) error { + config := &mapstructure.DecoderConfig{ + TagName: "json", + WeaklyTypedInput: true, + Result: result, + DecodeHook: mapstructure.ComposeDecodeHookFunc( + mapstructure.StringToTimeDurationHookFunc(), + mapstructure.StringToSliceHookFunc(","), + jsonUnmarshalerHookConfig, + ), + } + + decoder, err := mapstructure.NewDecoder(config) + if err != nil { + return err + } + + return decoder.Decode(input) +} + +func join_Config(arr interface{}, sep string) string { + listValue := reflect.ValueOf(arr) + strs := make([]string, 0, listValue.Len()) + for i := 0; i < listValue.Len(); i++ { + strs = append(strs, fmt.Sprintf("%v", listValue.Index(i))) + } + + return strings.Join(strs, sep) +} + +func testDecodeJson_Config(t *testing.T, val, result interface{}) { + assert.NoError(t, decode_Config(val, result)) +} + +func testDecodeRaw_Config(t *testing.T, vStringSlice, result interface{}) { + assert.NoError(t, decode_Config(vStringSlice, result)) +} + +func TestConfig_GetPFlagSet(t *testing.T) { + val := Config{} + cmdFlags := val.GetPFlagSet("") + assert.True(t, cmdFlags.HasFlags()) +} + +func TestConfig_SetFlags(t *testing.T) { + actual := Config{} + cmdFlags := actual.GetPFlagSet("") + assert.True(t, cmdFlags.HasFlags()) + + t.Run("Test_callback-uri", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("callback-uri", testValue) + if vString, err := cmdFlags.GetString("callback-uri"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.CallbackURI) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_endpoint", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("endpoint", testValue) + if vString, err := cmdFlags.GetString("endpoint"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Endpoint) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_env-detect-orphan-interval", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultConfig.EnvDetectOrphanInterval.String() + + cmdFlags.Set("env-detect-orphan-interval", testValue) + if vString, err := cmdFlags.GetString("env-detect-orphan-interval"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.EnvDetectOrphanInterval) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_env-gc-interval", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultConfig.EnvGCInterval.String() + + cmdFlags.Set("env-gc-interval", testValue) + if vString, err := cmdFlags.GetString("env-gc-interval"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.EnvGCInterval) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_env-repair-interval", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultConfig.EnvRepairInterval.String() + + cmdFlags.Set("env-repair-interval", testValue) + if vString, err := cmdFlags.GetString("env-repair-interval"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.EnvRepairInterval) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_grace-period-status-not-found", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultConfig.GracePeriodStatusNotFound.String() + + cmdFlags.Set("grace-period-status-not-found", testValue) + if vString, err := cmdFlags.GetString("grace-period-status-not-found"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.GracePeriodStatusNotFound) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_grace-period-workers-unavailable", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultConfig.GracePeriodWorkersUnavailable.String() + + cmdFlags.Set("grace-period-workers-unavailable", testValue) + if vString, err := cmdFlags.GetString("grace-period-workers-unavailable"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.GracePeriodWorkersUnavailable) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_heartbeat-buffer-size", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("heartbeat-buffer-size", testValue) + if vInt, err := cmdFlags.GetInt("heartbeat-buffer-size"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vInt), &actual.HeartbeatBufferSize) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_image", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("image", testValue) + if vString, err := cmdFlags.GetString("image"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.Image) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_init-container-cpu", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultConfig.InitContainerCPU.String() + + cmdFlags.Set("init-container-cpu", testValue) + if vString, err := cmdFlags.GetString("init-container-cpu"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.InitContainerCPU) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_init-container-memory", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := defaultConfig.InitContainerMemory.String() + + cmdFlags.Set("init-container-memory", testValue) + if vString, err := cmdFlags.GetString("init-container-memory"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.InitContainerMemory) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_nonce-length", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("nonce-length", testValue) + if vInt, err := cmdFlags.GetInt("nonce-length"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vInt), &actual.NonceLength) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) + t.Run("Test_task-status-buffer-size", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("task-status-buffer-size", testValue) + if vInt, err := cmdFlags.GetInt("task-status-buffer-size"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vInt), &actual.TaskStatusBufferSize) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) +} diff --git a/fasttask/plugin/go.mod b/fasttask/plugin/go.mod new file mode 100644 index 0000000000..ebf77a4808 --- /dev/null +++ b/fasttask/plugin/go.mod @@ -0,0 +1,124 @@ +module github.com/unionai/flyte/fasttask/plugin + +go 1.21 + +toolchain go1.21.4 + +require ( + github.com/flyteorg/flyte/flyteidl v0.0.0-00010101000000-000000000000 + github.com/flyteorg/flyte/flyteplugins v0.0.0-00010101000000-000000000000 + github.com/flyteorg/flyte/flytestdlib v0.0.0-00010101000000-000000000000 + github.com/golang/protobuf v1.5.3 + github.com/mitchellh/mapstructure v1.5.0 + github.com/spf13/pflag v1.0.5 + github.com/stretchr/testify v1.8.4 + google.golang.org/grpc v1.60.1 + google.golang.org/protobuf v1.32.0 + k8s.io/api v0.28.3 + k8s.io/apimachinery v0.28.3 +) + +require ( + cloud.google.com/go v0.110.8 // indirect + cloud.google.com/go/compute v1.23.0 // indirect + cloud.google.com/go/compute/metadata v0.2.3 // indirect + cloud.google.com/go/iam v1.1.2 // indirect + cloud.google.com/go/storage v1.30.1 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.2 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.1 // indirect + github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0 // indirect + github.com/AzureAD/microsoft-authentication-library-for-go v1.2.0 // indirect + github.com/aws/aws-sdk-go v1.44.2 // indirect + github.com/benlaurie/objecthash v0.0.0-20180202135721-d1e3d6079fc1 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash v1.1.0 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/coocood/freecache v1.1.1 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/emicklei/go-restful/v3 v3.11.0 // indirect + github.com/evanphx/json-patch/v5 v5.6.0 // indirect + github.com/fatih/color v1.13.0 // indirect + github.com/flyteorg/flyte/flytepropeller v0.0.0-00010101000000-000000000000 // indirect + github.com/flyteorg/stow v0.3.8 // indirect + github.com/ghodss/yaml v1.0.0 // indirect + github.com/go-logr/logr v1.4.1 // indirect + github.com/go-logr/stdr v1.2.2 // indirect + github.com/go-openapi/jsonpointer v0.19.6 // indirect + github.com/go-openapi/jsonreference v0.20.2 // indirect + github.com/go-openapi/swag v0.22.3 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang-jwt/jwt/v5 v5.0.0 // indirect + github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect + github.com/google/gnostic-models v0.6.8 // indirect + github.com/google/go-cmp v0.6.0 // indirect + github.com/google/gofuzz v1.2.0 // indirect + github.com/google/s2a-go v0.1.4 // indirect + github.com/google/uuid v1.3.1 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.2.4 // indirect + github.com/googleapis/gax-go/v2 v2.12.0 // indirect + github.com/hashicorp/golang-lru v0.5.4 // indirect + github.com/imdario/mergo v0.3.13 // indirect + github.com/inconshreveable/mousetrap v1.1.0 // indirect + github.com/jmespath/go-jmespath v0.4.0 // indirect + github.com/josharian/intern v1.0.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/kylelemons/godebug v1.1.0 // indirect + github.com/mailru/easyjson v0.7.7 // indirect + github.com/mattn/go-colorable v0.1.12 // indirect + github.com/mattn/go-isatty v0.0.16 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect + github.com/ncw/swift v1.0.53 // indirect + github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/prometheus/client_golang v1.16.0 // indirect + github.com/prometheus/client_model v0.4.0 // indirect + github.com/prometheus/common v0.44.0 // indirect + github.com/prometheus/procfs v0.10.1 // indirect + github.com/sirupsen/logrus v1.9.3 // indirect + github.com/spf13/cobra v1.7.0 // indirect + go.opencensus.io v0.24.0 // indirect + go.opentelemetry.io/otel v1.22.0 // indirect + go.opentelemetry.io/otel/exporters/jaeger v1.17.0 // indirect + go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.22.0 // indirect + go.opentelemetry.io/otel/metric v1.22.0 // indirect + go.opentelemetry.io/otel/sdk v1.22.0 // indirect + go.opentelemetry.io/otel/trace v1.22.0 // indirect + golang.org/x/crypto v0.18.0 // indirect + golang.org/x/exp v0.0.0-20231005195138-3e424a577f31 // indirect + golang.org/x/net v0.20.0 // indirect + golang.org/x/oauth2 v0.13.0 // indirect + golang.org/x/sync v0.4.0 // indirect + golang.org/x/sys v0.16.0 // indirect + golang.org/x/term v0.16.0 // indirect + golang.org/x/text v0.14.0 // indirect + golang.org/x/time v0.3.0 // indirect + golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect + google.golang.org/api v0.128.0 // indirect + google.golang.org/appengine v1.6.8 // indirect + google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect + gopkg.in/inf.v0 v0.9.1 // indirect + gopkg.in/yaml.v2 v2.4.0 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect + k8s.io/client-go v0.28.3 // indirect + k8s.io/klog/v2 v2.100.1 // indirect + k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9 // indirect + k8s.io/utils v0.0.0-20230406110748-d93618cff8a2 // indirect + sigs.k8s.io/controller-runtime v0.16.3 // indirect + sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd // indirect + sigs.k8s.io/structured-merge-diff/v4 v4.2.3 // indirect + sigs.k8s.io/yaml v1.3.0 // indirect +) + +replace ( + github.com/flyteorg/flyte/flyteidl => ../../flyte-union/flyteidl + github.com/flyteorg/flyte/flyteplugins => ../../flyte-union/flyteplugins + github.com/flyteorg/flyte/flytepropeller => ../../flyte-union/flytepropeller + github.com/flyteorg/flyte/flytestdlib => ../../flyte-union/flytestdlib +) diff --git a/fasttask/plugin/go.sum b/fasttask/plugin/go.sum new file mode 100644 index 0000000000..0a52d4abef --- /dev/null +++ b/fasttask/plugin/go.sum @@ -0,0 +1,461 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.110.8 h1:tyNdfIxjzaWctIiLYOTalaLKZ17SI44SKFW26QbOhME= +cloud.google.com/go v0.110.8/go.mod h1:Iz8AkXJf1qmxC3Oxoep8R1T36w8B92yU29PcBhHO5fk= +cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= +cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= +cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/iam v1.1.2 h1:gacbrBdWcoVmGLozRuStX45YKvJtzIjJdAolzUs1sm4= +cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= +cloud.google.com/go/storage v1.30.1 h1:uOdMxAs8HExqBlnLtnQyP0YkvbiDpdGShGKtx6U/oNM= +cloud.google.com/go/storage v1.30.1/go.mod h1:NfxhC0UJE1aXSx7CIIbCf7y9HKT7BiccwkR7+P7gN8E= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.2 h1:t5+QXLCK9SVi0PPdaY0PrFvYUo24KwA0QwxnaHRSVd4= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.2/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.1 h1:LNHhpdK7hzUcx/k1LIcuh5k7k1LGIWLQfCjaneSj7Fc= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.1/go.mod h1:uE9zaUfEQT/nbQjVi2IblCG9iaLtZsuYZ8ne+PuQ02M= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 h1:sXr+ck84g/ZlZUOZiNELInmMgOsuGwdjjVkEIde0OtY= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0/go.mod h1:okt5dMMTOFjX/aovMlrjvvXoPMBVSPzk9185BT0+eZM= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.2.0 h1:Ma67P/GGprNwsslzEH6+Kb8nybI8jpDTm4Wmzu2ReK8= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.2.0/go.mod h1:c+Lifp3EDEamAkPVzMooRNOK6CZjNSdEnf1A7jsI9u4= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0 h1:nVocQV40OQne5613EeLayJiRAJuKlBGy+m22qWG+WRg= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0/go.mod h1:7QJP7dr2wznCMeqIrhMgWGf7XpAQnVrJqDm9nvV3Cu4= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.0 h1:hVeq+yCyUi+MsoO/CU95yqCIcdzra5ovzk8Q2BBpV2M= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.0/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= +github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= +github.com/aws/aws-sdk-go v1.44.2 h1:5VBk5r06bgxgRKVaUtm1/4NT/rtrnH2E4cnAYv5zgQc= +github.com/aws/aws-sdk-go v1.44.2/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX4oIKwKHZo= +github.com/benlaurie/objecthash v0.0.0-20180202135721-d1e3d6079fc1 h1:VRtJdDi2lqc3MFwmouppm2jlm6icF+7H3WYKpLENMTo= +github.com/benlaurie/objecthash v0.0.0-20180202135721-d1e3d6079fc1/go.mod h1:jvdWlw8vowVGnZqSDC7yhPd7AifQeQbRDkZcQXV2nRg= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= +github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cheekybits/is v0.0.0-20150225183255-68e9c0620927 h1:SKI1/fuSdodxmNNyVBR8d7X/HuLnRpvvFO0AgyQk764= +github.com/cheekybits/is v0.0.0-20150225183255-68e9c0620927/go.mod h1:h/aW8ynjgkuj+NQRlZcDbAbM1ORAbXjXX77sX7T289U= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= +github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/coocood/freecache v1.1.1 h1:uukNF7QKCZEdZ9gAV7WQzvh0SbjwdMF6m3x3rxEkaPc= +github.com/coocood/freecache v1.1.1/go.mod h1:OKrEjkGVoxZhyWAJoeFi5BMLUJm2Tit0kpGkIr7NGYY= +github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= +github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= +github.com/emicklei/go-restful/v3 v3.11.0 h1:rAQeMHw1c7zTmncogyy8VvRZwtkmkZ4FxERmMY4rD+g= +github.com/emicklei/go-restful/v3 v3.11.0/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/evanphx/json-patch v5.6.0+incompatible h1:jBYDEEiFBPxA0v50tFdvOzQQTCvpL6mnFh5mB2/l16U= +github.com/evanphx/json-patch v5.6.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= +github.com/evanphx/json-patch/v5 v5.6.0 h1:b91NhWfaz02IuVxO9faSllyAtNXHMPkC5J8sJCLunww= +github.com/evanphx/json-patch/v5 v5.6.0/go.mod h1:G79N1coSVB93tBe7j6PhzjmR3/2VvlbKOFpnXhI9Bw4= +github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= +github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= +github.com/flyteorg/stow v0.3.8 h1:4a6BtfgDR86fUwa48DkkZTcp6WK4oQXSfewPd/kN0Z4= +github.com/flyteorg/stow v0.3.8/go.mod h1:fArjMpsYJNWkp/hyDKKdbcv07gxbuLmKFcb7YT1aSOM= +github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= +github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= +github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= +github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= +github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= +github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= +github.com/go-logr/zapr v1.2.4 h1:QHVo+6stLbfJmYGkQ7uGHUCu5hnAFAj6mDe6Ea0SeOo= +github.com/go-logr/zapr v1.2.4/go.mod h1:FyHWQIzQORZ0QVE1BtVHv3cKtNLuXsbNLtpuhNapBOA= +github.com/go-openapi/jsonpointer v0.19.6 h1:eCs3fxoIi3Wh6vtgmLTOjdhSpiqphQ+DaPn38N2ZdrE= +github.com/go-openapi/jsonpointer v0.19.6/go.mod h1:osyAmYz/mB/C3I+WsTTSgw1ONzaLJoLCyoi6/zppojs= +github.com/go-openapi/jsonreference v0.20.2 h1:3sVjiK66+uXK/6oQ8xgcRKcFgQ5KXa2KvnJRumpMGbE= +github.com/go-openapi/jsonreference v0.20.2/go.mod h1:Bl1zwGIM8/wsvqjsOQLJ/SH+En5Ap4rVB5KVcIDZG2k= +github.com/go-openapi/swag v0.22.3 h1:yMBqmnQ0gyZvEb/+KzuWZOXgllrXT4SADYbvDaXHv/g= +github.com/go-openapi/swag v0.22.3/go.mod h1:UzaqsxGiab7freDnrUUra0MwWfN/q7tE4j+VcZ0yl14= +github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 h1:tfuBGBXKqDEevZMzYi5KSi8KkcZtzBcTgAUUtapy0OI= +github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572/go.mod h1:9Pwr4B2jHnOSGXyyzV8ROjYa2ojvAY6HCGYYfMoC3Ls= +github.com/go-test/deep v1.0.7 h1:/VSMRlnY/JSyqxQUzQLKVMAskpY/NZKFA5j2P+0pP2M= +github.com/go-test/deep v1.0.7/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang-jwt/jwt/v5 v5.0.0 h1:1n1XNM9hk7O9mnQoNBGolZvzebBQ7p93ULHRc28XJUE= +github.com/golang-jwt/jwt/v5 v5.0.0/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/google/gnostic-models v0.6.8 h1:yo/ABAfM5IMRsS1VnXjTBvUb61tFIHozhlYvRgGre9I= +github.com/google/gnostic-models v0.6.8/go.mod h1:5n7qKqH0f5wFt+aWF8CW6pZLLNOfYuF5OpfBSENuI8U= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= +github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= +github.com/google/martian/v3 v3.3.2/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= +github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1 h1:K6RDEckDVWvDI9JAJYCmNdQXq6neHJOYx3V6jnqNEec= +github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/s2a-go v0.1.4 h1:1kZ/sQM3srePvKs3tXAvQzo66XfcReoqFpIpIccE7Oc= +github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/enterprise-certificate-proxy v0.2.4 h1:uGy6JWR/uMIILU8wbf+OkstIrNiMjGpEIyhx8f6W7s4= +github.com/googleapis/enterprise-certificate-proxy v0.2.4/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= +github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= +github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= +github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4= +github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/imdario/mergo v0.3.13 h1:lFzP57bqS/wsqKssCGmtLAb8A0wKjLGrve2q3PPVcBk= +github.com/imdario/mergo v0.3.13/go.mod h1:4lJ1jqUDcsbIECGy0RUJAXNIhg+6ocWgb1ALK2O4oXg= +github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= +github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= +github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= +github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= +github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= +github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= +github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= +github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= +github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= +github.com/magiconair/properties v1.8.6 h1:5ibWZ6iY0NctNGWo87LalDlEZ6R41TqbbDamhfG/Qzo= +github.com/magiconair/properties v1.8.6/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= +github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0= +github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= +github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= +github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= +github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.16 h1:bq3VjFmv/sOjHtdEhmkEV4x1AJtvUvOJ2PFAZ5+peKQ= +github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= +github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= +github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/ncw/swift v1.0.53 h1:luHjjTNtekIEvHg5KdAFIBaH7bWfNkefwFnpDffSIks= +github.com/ncw/swift v1.0.53/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= +github.com/onsi/ginkgo/v2 v2.11.0 h1:WgqUCUt/lT6yXoQ8Wef0fsNn5cAuMK7+KT9UFRz2tcU= +github.com/onsi/ginkgo/v2 v2.11.0/go.mod h1:ZhrRA5XmEE3x3rhlzamx/JJvujdZoJ2uvgI7kR0iZvM= +github.com/onsi/gomega v1.27.10 h1:naR28SdDFlqrG6kScpT8VWpu1xWY5nJRCF3XaYyBjhI= +github.com/onsi/gomega v1.27.10/go.mod h1:RsS8tutOdbdgzbPtzzATp12yT7kM5I5aElG3evPbQ0M= +github.com/pelletier/go-toml v1.9.4 h1:tjENF6MfZAg8e4ZmZTeWaWiT2vXtsoO6+iuOjFhECwM= +github.com/pelletier/go-toml v1.9.4/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= +github.com/pelletier/go-toml/v2 v2.0.0-beta.8 h1:dy81yyLYJDwMTifq24Oi/IslOslRrDSb3jwDggjz3Z0= +github.com/pelletier/go-toml/v2 v2.0.0-beta.8/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= +github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 h1:KoWmjvw+nsYOo29YJK9vDA65RGE3NrOnUtO7a+RF9HU= +github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8/go.mod h1:HKlIX3XHQyzLZPlr7++PzdhaXEj94dEiJgZDTsxEqUI= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= +github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= +github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= +github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= +github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= +github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= +github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= +github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= +github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= +github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= +github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= +github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/afero v1.9.2 h1:j49Hj62F0n+DaZ1dDCvhABaPNSGNkt32oRFxI33IEMw= +github.com/spf13/afero v1.9.2/go.mod h1:iUV7ddyEEZPO5gA3zD4fJt6iStLlL+Lg4m2cihcDf8Y= +github.com/spf13/cast v1.4.1 h1:s0hze+J0196ZfEMTs80N7UlFt0BDuQ7Q+JDnHiMWKdA= +github.com/spf13/cast v1.4.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +github.com/spf13/cobra v1.7.0 h1:hyqWnYt1ZQShIddO5kBpj3vu05/++x6tJ6dg8EC572I= +github.com/spf13/cobra v1.7.0/go.mod h1:uLxZILRyS/50WlhOIKD7W6V5bgeIt+4sICxh6uRMrb0= +github.com/spf13/jwalterweatherman v1.1.0 h1:ue6voC5bR5F8YxI5S67j9i582FU4Qvo2bmqnqMYADFk= +github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= +github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/viper v1.11.0 h1:7OX/1FS6n7jHD1zGrZTM7WtY13ZELRyosK4k93oPr44= +github.com/spf13/viper v1.11.0/go.mod h1:djo0X/bA5+tYVoCn+C7cAYJGcVn/qYLFTG8gdUsX7Zk= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/subosito/gotenv v1.2.0 h1:Slr1R9HxAlEKefgq5jn9U+DnETlIUa6HfgEzj0g5d7s= +github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= +go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= +go.opentelemetry.io/otel v1.22.0 h1:xS7Ku+7yTFvDfDraDIJVpw7XPyuHlB9MCiqqX5mcJ6Y= +go.opentelemetry.io/otel v1.22.0/go.mod h1:eoV4iAi3Ea8LkAEI9+GFT44O6T/D0GWAVFyZVCC6pMI= +go.opentelemetry.io/otel/exporters/jaeger v1.17.0 h1:D7UpUy2Xc2wsi1Ras6V40q806WM07rqoCWzXu7Sqy+4= +go.opentelemetry.io/otel/exporters/jaeger v1.17.0/go.mod h1:nPCqOnEH9rNLKqH/+rrUjiMzHJdV1BlpKcTwRTyKkKI= +go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.22.0 h1:zr8ymM5OWWjjiWRzwTfZ67c905+2TMHYp2lMJ52QTyM= +go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.22.0/go.mod h1:sQs7FT2iLVJ+67vYngGJkPe1qr39IzaBzaj9IDNNY8k= +go.opentelemetry.io/otel/metric v1.22.0 h1:lypMQnGyJYeuYPhOM/bgjbFM6WE44W1/T45er4d8Hhg= +go.opentelemetry.io/otel/metric v1.22.0/go.mod h1:evJGjVpZv0mQ5QBRJoBF64yMuOf4xCWdXjK8pzFvliY= +go.opentelemetry.io/otel/sdk v1.22.0 h1:6coWHw9xw7EfClIC/+O31R8IY3/+EiRFHevmHafB2Gw= +go.opentelemetry.io/otel/sdk v1.22.0/go.mod h1:iu7luyVGYovrRpe2fmj3CVKouQNdTOkxtLzPvPz1DOc= +go.opentelemetry.io/otel/trace v1.22.0 h1:Hg6pPujv0XG9QaVbGOBVHunyuLcCC3jN7WEhPx83XD0= +go.opentelemetry.io/otel/trace v1.22.0/go.mod h1:RbbHXVqKES9QhzZq/fE5UnOSILqRt40a21sPw2He1xo= +go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= +go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= +go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/zap v1.25.0 h1:4Hvk6GtkucQ790dqmj7l1eEnRdKm3k3ZUrUMS2d5+5c= +go.uber.org/zap v1.25.0/go.mod h1:JIAUzQIH94IC4fOJQm7gMmBJP5k7wQfdcnYdPoEXJYk= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= +golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20231005195138-3e424a577f31 h1:9k5exFQKQglLo+RoP+4zMjOFE14P6+vyR0baDAi0Rcs= +golang.org/x/exp v0.0.0-20231005195138-3e424a577f31/go.mod h1:S2oDrQGGwySpoQPVqRShND87VCbxmc6bL1Yd2oYrm6k= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= +golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.13.0 h1:jDDenyj+WgFtmV3zYVoi8aE2BwtXFLWOA67ZfNWftiY= +golang.org/x/oauth2 v0.13.0/go.mod h1:/JMhi4ZRXAf4HG9LiNmxvk+45+96RUlVThiH8FzNBn0= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= +golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210616045830-e2b7044e8c71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.16.0 h1:xWw16ngr6ZMtmxDyKyIgsE93KNKz5HKmMa3b8ALHidU= +golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.16.0 h1:m+B6fahuftsE9qjo0VWp2FW0mB3MTJvR0BaMQrq0pmE= +golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/tools v0.13.0 h1:Iey4qkscZuv0VvIt8E0neZjtPVQFSc870HQ448QgEmQ= +golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 h1:H2TDz8ibqkAF6YGhCdN3jS9O0/s90v0rJh3X/OLHEUk= +golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= +gomodules.xyz/jsonpatch/v2 v2.4.0 h1:Ci3iUJyx9UeRx7CeFN8ARgGbkESwJK+KB9lLcWxY/Zw= +gomodules.xyz/jsonpatch/v2 v2.4.0/go.mod h1:AH3dM2RI6uoBZxn3LVrfvJ3E0/9dG4cSrbuBJT4moAY= +google.golang.org/api v0.128.0 h1:RjPESny5CnQRn9V6siglged+DZCgfu9l6mO9dkX9VOg= +google.golang.org/api v0.128.0/go.mod h1:Y611qgqaE92On/7g65MQgxYul3c0rEB894kniWLY750= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= +google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97 h1:SeZZZx0cP0fqUyA+oRzP9k7cSwJlvDFiROO72uwD6i0= +google.golang.org/genproto v0.0.0-20231002182017-d307bd883b97/go.mod h1:t1VqOqqvce95G3hIDCT5FeO3YUc6Q4Oe24L/+rNMxRk= +google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97 h1:W18sezcAYs+3tDZX4F80yctqa12jcP1PUS2gQu1zTPU= +google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97/go.mod h1:iargEX0SFPm3xcfMI0d1domjg0ZF4Aa0p2awqyxhvF0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 h1:6GQBEOdGkX6MMTLT9V+TjtIRZCw9VPD5Z+yHY9wMgS0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97/go.mod h1:v7nGkzlmW8P3n/bKmWBn2WpBjpOEx8Q6gMueudAmKfY= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= +google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= +google.golang.org/grpc v1.60.1 h1:26+wFr+cNqSGFcOXcabYC0lUVJVRa2Sb2ortSK7VrEU= +google.golang.org/grpc v1.60.1/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= +google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= +gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= +gopkg.in/ini.v1 v1.66.4 h1:SsAcf+mM7mRZo2nJNGt8mZCjG8ZRaNGMURJw7BsIST4= +gopkg.in/ini.v1 v1.66.4/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo= +gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +k8s.io/api v0.28.3 h1:Gj1HtbSdB4P08C8rs9AR94MfSGpRhJgsS+GF9V26xMM= +k8s.io/api v0.28.3/go.mod h1:MRCV/jr1dW87/qJnZ57U5Pak65LGmQVkKTzf3AtKFHc= +k8s.io/apiextensions-apiserver v0.28.3 h1:Od7DEnhXHnHPZG+W9I97/fSQkVpVPQx2diy+2EtmY08= +k8s.io/apiextensions-apiserver v0.28.3/go.mod h1:NE1XJZ4On0hS11aWWJUTNkmVB03j9LM7gJSisbRt8Lc= +k8s.io/apimachinery v0.28.3 h1:B1wYx8txOaCQG0HmYF6nbpU8dg6HvA06x5tEffvOe7A= +k8s.io/apimachinery v0.28.3/go.mod h1:uQTKmIqs+rAYaq+DFaoD2X7pcjLOqbQX2AOiO0nIpb8= +k8s.io/client-go v0.28.3 h1:2OqNb72ZuTZPKCl+4gTKvqao0AMOl9f3o2ijbAj3LI4= +k8s.io/client-go v0.28.3/go.mod h1:LTykbBp9gsA7SwqirlCXBWtK0guzfhpoW4qSm7i9dxo= +k8s.io/component-base v0.28.3 h1:rDy68eHKxq/80RiMb2Ld/tbH8uAE75JdCqJyi6lXMzI= +k8s.io/component-base v0.28.3/go.mod h1:fDJ6vpVNSk6cRo5wmDa6eKIG7UlIQkaFmZN2fYgIUD8= +k8s.io/klog/v2 v2.100.1 h1:7WCHKK6K8fNhTqfBhISHQ97KrnJNFZMcQvKp7gP/tmg= +k8s.io/klog/v2 v2.100.1/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0= +k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9 h1:LyMgNKD2P8Wn1iAwQU5OhxCKlKJy0sHc+PcDwFB24dQ= +k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9/go.mod h1:wZK2AVp1uHCp4VamDVgBP2COHZjqD1T68Rf0CM3YjSM= +k8s.io/utils v0.0.0-20230406110748-d93618cff8a2 h1:qY1Ad8PODbnymg2pRbkyMT/ylpTrCM8P2RJ0yroCyIk= +k8s.io/utils v0.0.0-20230406110748-d93618cff8a2/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= +sigs.k8s.io/controller-runtime v0.16.3 h1:2TuvuokmfXvDUamSx1SuAOO3eTyye+47mJCigwG62c4= +sigs.k8s.io/controller-runtime v0.16.3/go.mod h1:j7bialYoSn142nv9sCOJmQgDXQXxnroFU4VnX/brVJ0= +sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMmr1bNJefnuqLsRAsHZo= +sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= +sigs.k8s.io/structured-merge-diff/v4 v4.2.3 h1:PRbqxJClWWYMNV1dhaG4NsibJbArud9kFxnAMREiWFE= +sigs.k8s.io/structured-merge-diff/v4 v4.2.3/go.mod h1:qjx8mGObPmV2aSZepjQjbmb2ihdVs8cGKBraizNC69E= +sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= +sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= diff --git a/fasttask/plugin/pb/fasttask.pb.go b/fasttask/plugin/pb/fasttask.pb.go new file mode 100644 index 0000000000..207b1f5ab4 --- /dev/null +++ b/fasttask/plugin/pb/fasttask.pb.go @@ -0,0 +1,747 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.1 +// protoc (unknown) +// source: fasttask.proto + +package pb + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + _ "google.golang.org/protobuf/types/known/structpb" + reflect "reflect" + sync "sync" +) + +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 HeartbeatResponse_Operation int32 + +const ( + HeartbeatResponse_ACK HeartbeatResponse_Operation = 0 + HeartbeatResponse_ASSIGN HeartbeatResponse_Operation = 1 + HeartbeatResponse_DELETE HeartbeatResponse_Operation = 2 +) + +// Enum value maps for HeartbeatResponse_Operation. +var ( + HeartbeatResponse_Operation_name = map[int32]string{ + 0: "ACK", + 1: "ASSIGN", + 2: "DELETE", + } + HeartbeatResponse_Operation_value = map[string]int32{ + "ACK": 0, + "ASSIGN": 1, + "DELETE": 2, + } +) + +func (x HeartbeatResponse_Operation) Enum() *HeartbeatResponse_Operation { + p := new(HeartbeatResponse_Operation) + *p = x + return p +} + +func (x HeartbeatResponse_Operation) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (HeartbeatResponse_Operation) Descriptor() protoreflect.EnumDescriptor { + return file_fasttask_proto_enumTypes[0].Descriptor() +} + +func (HeartbeatResponse_Operation) Type() protoreflect.EnumType { + return &file_fasttask_proto_enumTypes[0] +} + +func (x HeartbeatResponse_Operation) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use HeartbeatResponse_Operation.Descriptor instead. +func (HeartbeatResponse_Operation) EnumDescriptor() ([]byte, []int) { + return file_fasttask_proto_rawDescGZIP(), []int{3, 0} +} + +type TaskStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskId string `protobuf:"bytes,1,opt,name=taskId,proto3" json:"taskId,omitempty"` + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowId string `protobuf:"bytes,3,opt,name=workflowId,proto3" json:"workflowId,omitempty"` + Phase int32 `protobuf:"varint,4,opt,name=phase,proto3" json:"phase,omitempty"` + Reason string `protobuf:"bytes,5,opt,name=reason,proto3" json:"reason,omitempty"` +} + +func (x *TaskStatus) Reset() { + *x = TaskStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_fasttask_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TaskStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskStatus) ProtoMessage() {} + +func (x *TaskStatus) ProtoReflect() protoreflect.Message { + mi := &file_fasttask_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 TaskStatus.ProtoReflect.Descriptor instead. +func (*TaskStatus) Descriptor() ([]byte, []int) { + return file_fasttask_proto_rawDescGZIP(), []int{0} +} + +func (x *TaskStatus) GetTaskId() string { + if x != nil { + return x.TaskId + } + return "" +} + +func (x *TaskStatus) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *TaskStatus) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *TaskStatus) GetPhase() int32 { + if x != nil { + return x.Phase + } + return 0 +} + +func (x *TaskStatus) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +type Capacity struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ExecutionCount int32 `protobuf:"varint,1,opt,name=executionCount,proto3" json:"executionCount,omitempty"` + ExecutionLimit int32 `protobuf:"varint,2,opt,name=executionLimit,proto3" json:"executionLimit,omitempty"` + BacklogCount int32 `protobuf:"varint,3,opt,name=backlogCount,proto3" json:"backlogCount,omitempty"` + BacklogLimit int32 `protobuf:"varint,4,opt,name=backlogLimit,proto3" json:"backlogLimit,omitempty"` +} + +func (x *Capacity) Reset() { + *x = Capacity{} + if protoimpl.UnsafeEnabled { + mi := &file_fasttask_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Capacity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Capacity) ProtoMessage() {} + +func (x *Capacity) ProtoReflect() protoreflect.Message { + mi := &file_fasttask_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 Capacity.ProtoReflect.Descriptor instead. +func (*Capacity) Descriptor() ([]byte, []int) { + return file_fasttask_proto_rawDescGZIP(), []int{1} +} + +func (x *Capacity) GetExecutionCount() int32 { + if x != nil { + return x.ExecutionCount + } + return 0 +} + +func (x *Capacity) GetExecutionLimit() int32 { + if x != nil { + return x.ExecutionLimit + } + return 0 +} + +func (x *Capacity) GetBacklogCount() int32 { + if x != nil { + return x.BacklogCount + } + return 0 +} + +func (x *Capacity) GetBacklogLimit() int32 { + if x != nil { + return x.BacklogLimit + } + return 0 +} + +type HeartbeatRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + WorkerId string `protobuf:"bytes,1,opt,name=workerId,proto3" json:"workerId,omitempty"` + QueueId string `protobuf:"bytes,2,opt,name=queueId,proto3" json:"queueId,omitempty"` + Capacity *Capacity `protobuf:"bytes,3,opt,name=capacity,proto3" json:"capacity,omitempty"` + TaskStatuses []*TaskStatus `protobuf:"bytes,4,rep,name=taskStatuses,proto3" json:"taskStatuses,omitempty"` +} + +func (x *HeartbeatRequest) Reset() { + *x = HeartbeatRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_fasttask_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HeartbeatRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HeartbeatRequest) ProtoMessage() {} + +func (x *HeartbeatRequest) ProtoReflect() protoreflect.Message { + mi := &file_fasttask_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 HeartbeatRequest.ProtoReflect.Descriptor instead. +func (*HeartbeatRequest) Descriptor() ([]byte, []int) { + return file_fasttask_proto_rawDescGZIP(), []int{2} +} + +func (x *HeartbeatRequest) GetWorkerId() string { + if x != nil { + return x.WorkerId + } + return "" +} + +func (x *HeartbeatRequest) GetQueueId() string { + if x != nil { + return x.QueueId + } + return "" +} + +func (x *HeartbeatRequest) GetCapacity() *Capacity { + if x != nil { + return x.Capacity + } + return nil +} + +func (x *HeartbeatRequest) GetTaskStatuses() []*TaskStatus { + if x != nil { + return x.TaskStatuses + } + return nil +} + +type HeartbeatResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskId string `protobuf:"bytes,1,opt,name=taskId,proto3" json:"taskId,omitempty"` + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + WorkflowId string `protobuf:"bytes,3,opt,name=workflowId,proto3" json:"workflowId,omitempty"` + Cmd []string `protobuf:"bytes,4,rep,name=cmd,proto3" json:"cmd,omitempty"` + Operation HeartbeatResponse_Operation `protobuf:"varint,5,opt,name=operation,proto3,enum=fasttask.HeartbeatResponse_Operation" json:"operation,omitempty"` +} + +func (x *HeartbeatResponse) Reset() { + *x = HeartbeatResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_fasttask_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HeartbeatResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HeartbeatResponse) ProtoMessage() {} + +func (x *HeartbeatResponse) ProtoReflect() protoreflect.Message { + mi := &file_fasttask_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 HeartbeatResponse.ProtoReflect.Descriptor instead. +func (*HeartbeatResponse) Descriptor() ([]byte, []int) { + return file_fasttask_proto_rawDescGZIP(), []int{3} +} + +func (x *HeartbeatResponse) GetTaskId() string { + if x != nil { + return x.TaskId + } + return "" +} + +func (x *HeartbeatResponse) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *HeartbeatResponse) GetWorkflowId() string { + if x != nil { + return x.WorkflowId + } + return "" +} + +func (x *HeartbeatResponse) GetCmd() []string { + if x != nil { + return x.Cmd + } + return nil +} + +func (x *HeartbeatResponse) GetOperation() HeartbeatResponse_Operation { + if x != nil { + return x.Operation + } + return HeartbeatResponse_ACK +} + +type FastTaskEnvironment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + QueueId string `protobuf:"bytes,1,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"` +} + +func (x *FastTaskEnvironment) Reset() { + *x = FastTaskEnvironment{} + if protoimpl.UnsafeEnabled { + mi := &file_fasttask_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FastTaskEnvironment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FastTaskEnvironment) ProtoMessage() {} + +func (x *FastTaskEnvironment) ProtoReflect() protoreflect.Message { + mi := &file_fasttask_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 FastTaskEnvironment.ProtoReflect.Descriptor instead. +func (*FastTaskEnvironment) Descriptor() ([]byte, []int) { + return file_fasttask_proto_rawDescGZIP(), []int{4} +} + +func (x *FastTaskEnvironment) GetQueueId() string { + if x != nil { + return x.QueueId + } + return "" +} + +type FastTaskEnvironmentSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BacklogLength int32 `protobuf:"varint,1,opt,name=backlog_length,json=backlogLength,proto3" json:"backlog_length,omitempty"` + Parallelism int32 `protobuf:"varint,2,opt,name=parallelism,proto3" json:"parallelism,omitempty"` + PodTemplateSpec []byte `protobuf:"bytes,3,opt,name=pod_template_spec,json=podTemplateSpec,proto3" json:"pod_template_spec,omitempty"` + PrimaryContainerName string `protobuf:"bytes,4,opt,name=primary_container_name,json=primaryContainerName,proto3" json:"primary_container_name,omitempty"` + ReplicaCount int32 `protobuf:"varint,5,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` + // Types that are assignable to TerminationCriteria: + // + // *FastTaskEnvironmentSpec_TtlSeconds + TerminationCriteria isFastTaskEnvironmentSpec_TerminationCriteria `protobuf_oneof:"termination_criteria"` +} + +func (x *FastTaskEnvironmentSpec) Reset() { + *x = FastTaskEnvironmentSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_fasttask_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FastTaskEnvironmentSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FastTaskEnvironmentSpec) ProtoMessage() {} + +func (x *FastTaskEnvironmentSpec) ProtoReflect() protoreflect.Message { + mi := &file_fasttask_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 FastTaskEnvironmentSpec.ProtoReflect.Descriptor instead. +func (*FastTaskEnvironmentSpec) Descriptor() ([]byte, []int) { + return file_fasttask_proto_rawDescGZIP(), []int{5} +} + +func (x *FastTaskEnvironmentSpec) GetBacklogLength() int32 { + if x != nil { + return x.BacklogLength + } + return 0 +} + +func (x *FastTaskEnvironmentSpec) GetParallelism() int32 { + if x != nil { + return x.Parallelism + } + return 0 +} + +func (x *FastTaskEnvironmentSpec) GetPodTemplateSpec() []byte { + if x != nil { + return x.PodTemplateSpec + } + return nil +} + +func (x *FastTaskEnvironmentSpec) GetPrimaryContainerName() string { + if x != nil { + return x.PrimaryContainerName + } + return "" +} + +func (x *FastTaskEnvironmentSpec) GetReplicaCount() int32 { + if x != nil { + return x.ReplicaCount + } + return 0 +} + +func (m *FastTaskEnvironmentSpec) GetTerminationCriteria() isFastTaskEnvironmentSpec_TerminationCriteria { + if m != nil { + return m.TerminationCriteria + } + return nil +} + +func (x *FastTaskEnvironmentSpec) GetTtlSeconds() int32 { + if x, ok := x.GetTerminationCriteria().(*FastTaskEnvironmentSpec_TtlSeconds); ok { + return x.TtlSeconds + } + return 0 +} + +type isFastTaskEnvironmentSpec_TerminationCriteria interface { + isFastTaskEnvironmentSpec_TerminationCriteria() +} + +type FastTaskEnvironmentSpec_TtlSeconds struct { + TtlSeconds int32 `protobuf:"varint,6,opt,name=ttl_seconds,json=ttlSeconds,proto3,oneof"` +} + +func (*FastTaskEnvironmentSpec_TtlSeconds) isFastTaskEnvironmentSpec_TerminationCriteria() {} + +var File_fasttask_proto protoreflect.FileDescriptor + +var file_fasttask_proto_rawDesc = []byte{ + 0x0a, 0x0e, 0x66, 0x61, 0x73, 0x74, 0x74, 0x61, 0x73, 0x6b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x12, 0x08, 0x66, 0x61, 0x73, 0x74, 0x74, 0x61, 0x73, 0x6b, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, + 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x90, 0x01, 0x0a, 0x0a, 0x54, 0x61, 0x73, + 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, + 0x1c, 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, 0x12, 0x1e, 0x0a, + 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x12, 0x14, 0x0a, + 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x70, 0x68, + 0x61, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x22, 0xa2, 0x01, 0x0a, 0x08, + 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x26, 0x0a, 0x0e, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, + 0x12, 0x26, 0x0a, 0x0e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x6d, + 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x62, 0x61, 0x63, 0x6b, + 0x6c, 0x6f, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, + 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x22, 0x0a, 0x0c, + 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x0c, 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x4c, 0x69, 0x6d, 0x69, 0x74, + 0x22, 0xb2, 0x01, 0x0a, 0x10, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, + 0x64, 0x12, 0x18, 0x0a, 0x07, 0x71, 0x75, 0x65, 0x75, 0x65, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x71, 0x75, 0x65, 0x75, 0x65, 0x49, 0x64, 0x12, 0x2e, 0x0a, 0x08, 0x63, + 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x66, 0x61, 0x73, 0x74, 0x74, 0x61, 0x73, 0x6b, 0x2e, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, + 0x79, 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x38, 0x0a, 0x0c, 0x74, + 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x61, 0x73, 0x74, 0x74, 0x61, 0x73, 0x6b, 0x2e, 0x54, 0x61, 0x73, + 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0c, 0x74, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x65, 0x73, 0x22, 0xee, 0x01, 0x0a, 0x11, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, + 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, + 0x61, 0x73, 0x6b, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, + 0x6b, 0x49, 0x64, 0x12, 0x1c, 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, 0x12, 0x1e, 0x0a, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, + 0x64, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x6d, 0x64, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, + 0x63, 0x6d, 0x64, 0x12, 0x43, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x66, 0x61, 0x73, 0x74, 0x74, 0x61, 0x73, + 0x6b, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x2c, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x43, 0x4b, 0x10, 0x00, 0x12, 0x0a, + 0x0a, 0x06, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, + 0x4c, 0x45, 0x54, 0x45, 0x10, 0x02, 0x22, 0x30, 0x0a, 0x13, 0x46, 0x61, 0x73, 0x74, 0x54, 0x61, + 0x73, 0x6b, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x19, 0x0a, + 0x08, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x71, 0x75, 0x65, 0x75, 0x65, 0x49, 0x64, 0x22, 0xa4, 0x02, 0x0a, 0x17, 0x46, 0x61, 0x73, + 0x74, 0x54, 0x61, 0x73, 0x6b, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x53, 0x70, 0x65, 0x63, 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x5f, + 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x62, 0x61, + 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x20, 0x0a, 0x0b, 0x70, + 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0b, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x12, 0x2a, 0x0a, + 0x11, 0x70, 0x6f, 0x64, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x70, + 0x65, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x70, 0x6f, 0x64, 0x54, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x34, 0x0a, 0x16, 0x70, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x70, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0b, 0x74, 0x74, 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0a, 0x74, 0x74, 0x6c, + 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x42, 0x16, 0x0a, 0x14, 0x74, 0x65, 0x72, 0x6d, 0x69, + 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x72, 0x69, 0x74, 0x65, 0x72, 0x69, 0x61, 0x32, + 0x54, 0x0a, 0x08, 0x46, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x48, 0x0a, 0x09, 0x48, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x1a, 0x2e, 0x66, 0x61, 0x73, 0x74, 0x74, + 0x61, 0x73, 0x6b, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x66, 0x61, 0x73, 0x74, 0x74, 0x61, 0x73, 0x6b, 0x2e, + 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x28, 0x01, 0x30, 0x01, 0x42, 0x05, 0x5a, 0x03, 0x70, 0x62, 0x2f, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_fasttask_proto_rawDescOnce sync.Once + file_fasttask_proto_rawDescData = file_fasttask_proto_rawDesc +) + +func file_fasttask_proto_rawDescGZIP() []byte { + file_fasttask_proto_rawDescOnce.Do(func() { + file_fasttask_proto_rawDescData = protoimpl.X.CompressGZIP(file_fasttask_proto_rawDescData) + }) + return file_fasttask_proto_rawDescData +} + +var file_fasttask_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_fasttask_proto_msgTypes = make([]protoimpl.MessageInfo, 6) +var file_fasttask_proto_goTypes = []interface{}{ + (HeartbeatResponse_Operation)(0), // 0: fasttask.HeartbeatResponse.Operation + (*TaskStatus)(nil), // 1: fasttask.TaskStatus + (*Capacity)(nil), // 2: fasttask.Capacity + (*HeartbeatRequest)(nil), // 3: fasttask.HeartbeatRequest + (*HeartbeatResponse)(nil), // 4: fasttask.HeartbeatResponse + (*FastTaskEnvironment)(nil), // 5: fasttask.FastTaskEnvironment + (*FastTaskEnvironmentSpec)(nil), // 6: fasttask.FastTaskEnvironmentSpec +} +var file_fasttask_proto_depIdxs = []int32{ + 2, // 0: fasttask.HeartbeatRequest.capacity:type_name -> fasttask.Capacity + 1, // 1: fasttask.HeartbeatRequest.taskStatuses:type_name -> fasttask.TaskStatus + 0, // 2: fasttask.HeartbeatResponse.operation:type_name -> fasttask.HeartbeatResponse.Operation + 3, // 3: fasttask.FastTask.Heartbeat:input_type -> fasttask.HeartbeatRequest + 4, // 4: fasttask.FastTask.Heartbeat:output_type -> fasttask.HeartbeatResponse + 4, // [4:5] is the sub-list for method output_type + 3, // [3:4] 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_fasttask_proto_init() } +func file_fasttask_proto_init() { + if File_fasttask_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_fasttask_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TaskStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_fasttask_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Capacity); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_fasttask_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HeartbeatRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_fasttask_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HeartbeatResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_fasttask_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FastTaskEnvironment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_fasttask_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FastTaskEnvironmentSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_fasttask_proto_msgTypes[5].OneofWrappers = []interface{}{ + (*FastTaskEnvironmentSpec_TtlSeconds)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_fasttask_proto_rawDesc, + NumEnums: 1, + NumMessages: 6, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_fasttask_proto_goTypes, + DependencyIndexes: file_fasttask_proto_depIdxs, + EnumInfos: file_fasttask_proto_enumTypes, + MessageInfos: file_fasttask_proto_msgTypes, + }.Build() + File_fasttask_proto = out.File + file_fasttask_proto_rawDesc = nil + file_fasttask_proto_goTypes = nil + file_fasttask_proto_depIdxs = nil +} diff --git a/fasttask/plugin/pb/fasttask_grpc.pb.go b/fasttask/plugin/pb/fasttask_grpc.pb.go new file mode 100644 index 0000000000..c69952d326 --- /dev/null +++ b/fasttask/plugin/pb/fasttask_grpc.pb.go @@ -0,0 +1,141 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.3.0 +// - protoc (unknown) +// source: fasttask.proto + +package pb + +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.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +const ( + FastTask_Heartbeat_FullMethodName = "/fasttask.FastTask/Heartbeat" +) + +// FastTaskClient is the client API for FastTask 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. +type FastTaskClient interface { + Heartbeat(ctx context.Context, opts ...grpc.CallOption) (FastTask_HeartbeatClient, error) +} + +type fastTaskClient struct { + cc grpc.ClientConnInterface +} + +func NewFastTaskClient(cc grpc.ClientConnInterface) FastTaskClient { + return &fastTaskClient{cc} +} + +func (c *fastTaskClient) Heartbeat(ctx context.Context, opts ...grpc.CallOption) (FastTask_HeartbeatClient, error) { + stream, err := c.cc.NewStream(ctx, &FastTask_ServiceDesc.Streams[0], FastTask_Heartbeat_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &fastTaskHeartbeatClient{stream} + return x, nil +} + +type FastTask_HeartbeatClient interface { + Send(*HeartbeatRequest) error + Recv() (*HeartbeatResponse, error) + grpc.ClientStream +} + +type fastTaskHeartbeatClient struct { + grpc.ClientStream +} + +func (x *fastTaskHeartbeatClient) Send(m *HeartbeatRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *fastTaskHeartbeatClient) Recv() (*HeartbeatResponse, error) { + m := new(HeartbeatResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// FastTaskServer is the server API for FastTask service. +// All implementations must embed UnimplementedFastTaskServer +// for forward compatibility +type FastTaskServer interface { + Heartbeat(FastTask_HeartbeatServer) error + mustEmbedUnimplementedFastTaskServer() +} + +// UnimplementedFastTaskServer must be embedded to have forward compatible implementations. +type UnimplementedFastTaskServer struct { +} + +func (UnimplementedFastTaskServer) Heartbeat(FastTask_HeartbeatServer) error { + return status.Errorf(codes.Unimplemented, "method Heartbeat not implemented") +} +func (UnimplementedFastTaskServer) mustEmbedUnimplementedFastTaskServer() {} + +// UnsafeFastTaskServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to FastTaskServer will +// result in compilation errors. +type UnsafeFastTaskServer interface { + mustEmbedUnimplementedFastTaskServer() +} + +func RegisterFastTaskServer(s grpc.ServiceRegistrar, srv FastTaskServer) { + s.RegisterService(&FastTask_ServiceDesc, srv) +} + +func _FastTask_Heartbeat_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(FastTaskServer).Heartbeat(&fastTaskHeartbeatServer{stream}) +} + +type FastTask_HeartbeatServer interface { + Send(*HeartbeatResponse) error + Recv() (*HeartbeatRequest, error) + grpc.ServerStream +} + +type fastTaskHeartbeatServer struct { + grpc.ServerStream +} + +func (x *fastTaskHeartbeatServer) Send(m *HeartbeatResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *fastTaskHeartbeatServer) Recv() (*HeartbeatRequest, error) { + m := new(HeartbeatRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// FastTask_ServiceDesc is the grpc.ServiceDesc for FastTask service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var FastTask_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "fasttask.FastTask", + HandlerType: (*FastTaskServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Heartbeat", + Handler: _FastTask_Heartbeat_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "fasttask.proto", +} diff --git a/fasttask/plugin/plugin.go b/fasttask/plugin/plugin.go new file mode 100644 index 0000000000..f00169a1fd --- /dev/null +++ b/fasttask/plugin/plugin.go @@ -0,0 +1,317 @@ +package plugin + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "net" + "time" + + "k8s.io/api/core/v1" + "google.golang.org/grpc" + + idlcore "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + flyteerrors "github.com/flyteorg/flyte/flyteplugins/go/tasks/errors" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core/template" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/flytek8s" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/ioutils" + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/utils" + "github.com/unionai/flyte/fasttask/plugin/pb" +) + +const fastTaskType = "fast-task" +var statusUpdateNotFoundError = errors.New("StatusUpdateNotFound") + +type Phase int + +const ( + PhaseNotStarted Phase = iota + PhaseRunning +) + +// State maintains the current status of the task execution. +type State struct { + Phase Phase + WorkerID string + LastUpdated time.Time +} + +// Plugin is a fast task plugin that offers task execution to a worker pool. +type Plugin struct { + fastTaskService *FastTaskService +} + +// GetID returns the unique identifier for the plugin. +func (p *Plugin) GetID() string { + return fastTaskType +} + +// GetProperties returns the properties of the plugin. +func (p *Plugin) GetProperties() core.PluginProperties { + return core.PluginProperties{} +} + +// getExecutionEnv retrieves the execution environment for the task. If the environment does not +// exist, it will create it. +// this is here because we wanted uniformity within `TaskExecutionContext` where functions simply +// return an interface rather than doing any actual work. alternatively, we could bury this within +// `NodeExecutionContext` so other `ExecutionEnvironment` plugins do not need to duplicate this. +func (p *Plugin) getExecutionEnv(ctx context.Context, tCtx core.TaskExecutionContext) (*pb.FastTaskEnvironment, error) { + taskTemplate, err := tCtx.TaskReader().Read(ctx) + if err != nil { + return nil, err + } + + executionEnv := &idlcore.ExecutionEnv{} + if err := utils.UnmarshalStruct(taskTemplate.Custom, executionEnv); err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to unmarshal environment") + } + + switch e := executionEnv.GetEnvironment().(type) { + case *idlcore.ExecutionEnv_Spec: + executionEnvClient := tCtx.GetExecutionEnvClient() + + // if environment already exists then return it + if environment := executionEnvClient.Get(ctx, executionEnv.Id); environment != nil { + fastTaskEnvironment := &pb.FastTaskEnvironment{} + if err := utils.UnmarshalStruct(environment, fastTaskEnvironment); err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to unmarshal environment client") + } + + return fastTaskEnvironment, nil + } + + // create environment + environmentSpec := e.Spec + + fastTaskEnvironmentSpec := &pb.FastTaskEnvironmentSpec{} + if err := utils.UnmarshalStruct(environmentSpec, fastTaskEnvironmentSpec); err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to unmarshal environment spec") + } + + // if podTemplateSpec is not popualated - then generate from tCtx + if len(fastTaskEnvironmentSpec.PodTemplateSpec) == 0 { + podSpec, objectMeta, primaryContainerName, err := flytek8s.ToK8sPodSpec(ctx, tCtx) + if err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to create environment") + } + + podTemplateSpec := &v1.PodTemplateSpec{ + ObjectMeta: *objectMeta, + Spec: *podSpec, + } + podTemplateSpec.SetNamespace(tCtx.TaskExecutionMetadata().GetNamespace()) + + // need to marshal as JSON to maintain container resources, proto serialization does + // not persist these settings for `PodSpec` + podTemplateSpecBytes, err := json.Marshal(podTemplateSpec) + if err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to marshal pod template spec") + } + + fastTaskEnvironmentSpec.PodTemplateSpec = podTemplateSpecBytes + if err := utils.MarshalStruct(fastTaskEnvironmentSpec, environmentSpec); err != nil { + return nil, fmt.Errorf("unable to marshal EnvironmentSpec [%v], Err: [%v]", fastTaskEnvironmentSpec, err.Error()) + } + + fastTaskEnvironmentSpec.PrimaryContainerName = primaryContainerName + } + + environment, err := executionEnvClient.Create(ctx, executionEnv.Id, environmentSpec) + if err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to create environment") + } + + fastTaskEnvironment := &pb.FastTaskEnvironment{} + if err := utils.UnmarshalStruct(environment, fastTaskEnvironment); err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to unmarshal environment extant") + } + + return fastTaskEnvironment, nil + case *idlcore.ExecutionEnv_Extant: + fastTaskEnvironment := &pb.FastTaskEnvironment{} + if err := utils.UnmarshalStruct(e.Extant, fastTaskEnvironment); err != nil { + return nil, flyteerrors.Wrapf(flyteerrors.BadTaskSpecification, err, "failed to unmarshal environment extant") + } + + return fastTaskEnvironment, nil + } + + return nil, nil +} + +// Handle is the main entrypoint for the plugin. It will offer the task to the worker pool and +// monitor the task until completion. +func (p *Plugin) Handle(ctx context.Context, tCtx core.TaskExecutionContext) (core.Transition, error) { + fastTaskEnvironment, err := p.getExecutionEnv(ctx, tCtx) + if err != nil { + return core.UnknownTransition, err + } + + // retrieve plugin state + pluginState := &State{} + if _, err := tCtx.PluginStateReader().Get(pluginState); err != nil { + return core.UnknownTransition, flyteerrors.Wrapf(flyteerrors.CorruptedPluginState, err, "Failed to read unmarshal custom state") + } + + taskID, err := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetGeneratedNameWith(0, 50) + if err != nil { + return core.UnknownTransition, err + } + + phaseInfo := core.PhaseInfoUndefined + switch pluginState.Phase { + case PhaseNotStarted: + // read task template + taskTemplate, err := tCtx.TaskReader().Read(ctx) + if err != nil { + return core.UnknownTransition, err + } + + taskContainer := taskTemplate.GetContainer() + if taskContainer == nil { + return core.UnknownTransition, flyteerrors.Errorf(flyteerrors.BadTaskSpecification, "unable to create container with no definition in TaskTemplate") + } + + templateParameters := template.Parameters{ + TaskExecMetadata: tCtx.TaskExecutionMetadata(), + Inputs: tCtx.InputReader(), + OutputPath: tCtx.OutputWriter(), + Task: tCtx.TaskReader(), + } + command, err := template.Render(ctx, taskContainer.GetArgs(), templateParameters) + if err != nil { + return core.UnknownTransition, err + } + + // offer the work to the queue + ownerID := tCtx.TaskExecutionMetadata().GetOwnerID() + workerID, err := p.fastTaskService.OfferOnQueue(ctx, fastTaskEnvironment.QueueId, taskID, ownerID.Namespace, ownerID.Name, command) + if err != nil { + return core.UnknownTransition, err + } + + if len(workerID) > 0 { + pluginState.Phase = PhaseRunning + pluginState.WorkerID = workerID + pluginState.LastUpdated = time.Now() + + phaseInfo = core.PhaseInfoRunning(core.DefaultPhaseVersion, nil) + } else { + if pluginState.LastUpdated.IsZero() { + pluginState.LastUpdated = time.Now() + } + + // fail if no worker available within grace period + if time.Now().Sub(pluginState.LastUpdated) > GetConfig().GracePeriodWorkersUnavailable.Duration { + phaseInfo = core.PhaseInfoSystemFailure("unknown", "timed out waiting for worker availability", nil) + } else { + phaseInfo = core.PhaseInfoNotReady(time.Now(), core.DefaultPhaseVersion, "no workers available") + } + } + case PhaseRunning: + // check the task status + phase, reason, err := p.fastTaskService.CheckStatus(ctx, taskID, fastTaskEnvironment.QueueId, pluginState.WorkerID) + + now := time.Now() + if err != nil && !errors.Is(err, statusUpdateNotFoundError) { + return core.UnknownTransition, err + } else if errors.Is(err, statusUpdateNotFoundError) && now.Sub(pluginState.LastUpdated) > GetConfig().GracePeriodStatusNotFound.Duration { + // if task has not been updated within the grace period we should abort + return core.DoTransition(core.PhaseInfoSystemRetryableFailure("unknown", "task status update not reported within grace period", nil)), nil + } else if phase == core.PhaseSuccess { + taskTemplate, err := tCtx.TaskReader().Read(ctx) + if err != nil { + return core.UnknownTransition, err + } + + // gather outputs if they exist + if taskTemplate.Interface != nil && taskTemplate.Interface.Outputs != nil && taskTemplate.Interface.Outputs.Variables != nil { + outputReader := ioutils.NewRemoteFileOutputReader(ctx, tCtx.DataStore(), tCtx.OutputWriter(), tCtx.MaxDatasetSizeBytes()) + tCtx.OutputWriter().Put(ctx, outputReader) + if err != nil { + return core.UnknownTransition, err + } + } + + phaseInfo = core.PhaseInfoSuccess(nil) + } else if phase == core.PhaseRetryableFailure { + return core.DoTransition(core.PhaseInfoRetryableFailure("unknown", reason, nil)), nil + } else { + pluginState.LastUpdated = now + phaseInfo = core.PhaseInfoRunning(core.DefaultPhaseVersion, nil) + } + } + + // update plugin state + if err := tCtx.PluginStateWriter().Put(0, pluginState); err != nil { + return core.UnknownTransition, err + } + + return core.DoTransition(phaseInfo), nil +} + +// Abort halts the specified task execution. +func (p *Plugin) Abort(ctx context.Context, tCtx core.TaskExecutionContext) error { + // halting an execution is handled through sending a `DELETE` to the worker, which kills any + // active executions. this is performed in the `Finalize` function which is _always_ called + // during any abort. if this logic changes, we will need to add a call to + // `fastTaskService.Cleanup` to ensure proper abort here. + return nil +} + +// Finalize is called when the task execution is complete, performing any necessary cleanup. +func (p *Plugin) Finalize(ctx context.Context, tCtx core.TaskExecutionContext) error { + fastTaskEnvironment, err := p.getExecutionEnv(ctx, tCtx) + if err != nil { + return err + } + + // retrieve plugin state + pluginState := &State{} + if _, err := tCtx.PluginStateReader().Get(pluginState); err != nil { + return flyteerrors.Wrapf(flyteerrors.CorruptedPluginState, err, "Failed to read unmarshal custom state") + } + + taskID, err := tCtx.TaskExecutionMetadata().GetTaskExecutionID().GetGeneratedNameWith(0, 50) + if err != nil { + return err + } + + return p.fastTaskService.Cleanup(ctx, taskID, fastTaskEnvironment.QueueId, pluginState.WorkerID) +} + +// init registers the plugin with the plugin machinery. +func init() { + pluginmachinery.PluginRegistry().RegisterCorePlugin( + core.PluginEntry{ + ID: fastTaskType, + RegisteredTaskTypes: []core.TaskType{fastTaskType}, + LoadPlugin: func(ctx context.Context, iCtx core.SetupContext) (core.Plugin, error) { + // open tcp listener + listener, err := net.Listen("tcp", GetConfig().Endpoint) + if err != nil { + return nil, err + } + + // create and start grpc server + fastTaskService := NewFastTaskService(iCtx.EnqueueOwner()) + go func() { + grpcServer := grpc.NewServer() + pb.RegisterFastTaskServer(grpcServer, fastTaskService) + if err := grpcServer.Serve(listener); err != nil { + panic("failed to start grpc fast task grpc server") + } + }() + + return &Plugin{ + fastTaskService: fastTaskService, + }, nil + }, + IsDefault: false, + }, + ) +} diff --git a/fasttask/plugin/service.go b/fasttask/plugin/service.go new file mode 100644 index 0000000000..5b6a31a3a4 --- /dev/null +++ b/fasttask/plugin/service.go @@ -0,0 +1,299 @@ +package plugin + +import ( + "context" + "errors" + "fmt" + "io" + "math/rand" + "sync" + + "k8s.io/apimachinery/pkg/types" + + "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" + "github.com/flyteorg/flyte/flytestdlib/logger" + "github.com/unionai/flyte/fasttask/plugin/pb" +) + +// FastTaskService is a gRPC service that manages assignment and management of task executions with +// respect to fasttask workers. +type FastTaskService struct { + pb.UnimplementedFastTaskServer + enqueueOwner core.EnqueueOwner + queues map[string]*Queue + queuesLock sync.RWMutex + taskStatusChannels sync.Map // map[string]chan *WorkerTaskStatus +} + +// Queue is a collection of Workers that are capable of executing similar tasks. +type Queue struct { + lock sync.RWMutex + workers map[string]*Worker +} + +// Worker represents a fasttask worker. +type Worker struct { + workerID string + capacity *pb.Capacity + responseChan chan<- *pb.HeartbeatResponse +} + +// workerTaskStatus represents the status of a task as reported by a worker. +type workerTaskStatus struct { + workerID string + taskStatus *pb.TaskStatus +} + +// Heartbeat is a gRPC stream that manages the heartbeat of a fasttask worker. This includes +// receiving task status updates and sending task assignments. +func (f *FastTaskService) Heartbeat(stream pb.FastTask_HeartbeatServer) error { + workerID := "" + + // recv initial heartbeat request + heartbeatRequest, err := stream.Recv() + if heartbeatRequest != nil { + workerID = heartbeatRequest.WorkerId + } + + if err == io.EOF || heartbeatRequest == nil { + logger.Debugf(context.Background(), "heartbeat stream closed for worker %s", workerID) + return nil + } else if err != nil { + return err + } + + logger.Debugf(context.Background(), "received initial heartbeat for worker %s", workerID) + + // create worker + responseChan := make(chan *pb.HeartbeatResponse, GetConfig().HeartbeatBufferSize) + worker := &Worker{ + workerID: workerID, + capacity: heartbeatRequest.Capacity, + responseChan: responseChan, + } + + // register worker with queue + f.queuesLock.Lock() + queue, exists := f.queues[heartbeatRequest.QueueId] + if !exists { + queue = &Queue{ + workers: make(map[string]*Worker), + } + f.queues[heartbeatRequest.QueueId] = queue + } + f.queuesLock.Unlock() + + queue.lock.Lock() + queue.workers[workerID] = worker + queue.lock.Unlock() + + // cleanup worker on exit + defer func() { + f.queuesLock.Lock() + queue, exists := f.queues[heartbeatRequest.QueueId] + if exists { + queue.lock.Lock() + delete(queue.workers, workerID) + if len(queue.workers) == 0 { + delete(f.queues, heartbeatRequest.QueueId) + } + queue.lock.Unlock() + } + f.queuesLock.Unlock() + }() + + // start go routine to handle heartbeat responses + go func() { + for { + select { + case message := <-responseChan: + if err := stream.Send(message); err != nil { + logger.Warnf(context.Background(), "failed to send heartbeat response %+v", message) + } + case <-stream.Context().Done(): + return + } + } + }() + + // handle heartbeat requests + for { + heartbeatRequest, err := stream.Recv() + if err == io.EOF || heartbeatRequest == nil { + logger.Debugf(context.Background(), "heartbeat stream closed for worker %s", workerID) + break; + } else if err != nil { + logger.Warnf(context.Background(), "failed to recv heartbeat request %+v", err) + continue + } + + // update worker capacity + queue.lock.Lock() + worker.capacity = heartbeatRequest.Capacity + queue.lock.Unlock() + + for _, taskStatus := range heartbeatRequest.TaskStatuses { + // if the taskContext exists then send the taskStatus to the statusChannel + // if it does not exist, then this plugin has restarted and we rely on the `CheckStatus` to create a new TaskContext. + // this is because if `CheckStatus` is called, then the task is active and will be cleaned up on completion. If we + // created it here, then a worker could be reporting a status for a task that has already completed and the TaskContext + // cleanup would require a separate GC process. + if taskStatusChannelResult, exists := f.taskStatusChannels.Load(taskStatus.TaskId); exists { + taskStatusChannel := taskStatusChannelResult.(chan *workerTaskStatus) + taskStatusChannel <- &workerTaskStatus{ + workerID: worker.workerID, + taskStatus: taskStatus, + } + } + + // if taskStatus is complete then enqueueOwner for fast feedback + phase := core.Phase(taskStatus.Phase) + if phase == core.PhaseSuccess || phase == core.PhaseRetryableFailure { + ownerID := types.NamespacedName{ + Namespace: taskStatus.Namespace, + Name: taskStatus.WorkflowId, + } + + if err := f.enqueueOwner(ownerID); err != nil { + logger.Warnf(context.Background(), "failed to enqueue owner for task %s: %+v", taskStatus.TaskId, err) + } + } + } + } + + return nil +} + +// OfferOnQueue offers a task to a worker on a specific queue. If no workers are available, an +// empty string is returned. +func (f *FastTaskService) OfferOnQueue(ctx context.Context, queueID, taskID, namespace, workflowID string, cmd []string) (string, error) { + f.queuesLock.RLock() + defer f.queuesLock.RUnlock() + + queue, exists := f.queues[queueID] + if !exists { + return "", nil // no workers available + } + + // retrieve random worker with capacity + queue.lock.Lock() + defer queue.lock.Unlock() + + preferredWorkers := make([]*Worker, 0) + acceptedWorkers := make([]*Worker, 0) + for _, worker := range queue.workers { + if worker.capacity.ExecutionLimit - worker.capacity.ExecutionCount > 0 { + preferredWorkers = append(preferredWorkers, worker) + } else if worker.capacity.BacklogLimit - worker.capacity.BacklogCount > 0 { + acceptedWorkers = append(acceptedWorkers, worker) + } + } + + var worker *Worker + if len(preferredWorkers) > 0 { + worker = preferredWorkers[rand.Intn(len(preferredWorkers))] + worker.capacity.ExecutionCount++ + } else if len(acceptedWorkers) > 0 { + worker = acceptedWorkers[rand.Intn(len(acceptedWorkers))] + worker.capacity.BacklogCount++ + } else { + return "", nil // no workers available + } + + // send assign message to worker + worker.responseChan <- &pb.HeartbeatResponse{ + TaskId: taskID, + Namespace: namespace, + WorkflowId: workflowID, + Cmd: cmd, + Operation: pb.HeartbeatResponse_ASSIGN, + } + + // create task status channel + f.taskStatusChannels.Store(taskID, make(chan *workerTaskStatus, GetConfig().TaskStatusBufferSize)) + return worker.workerID, nil +} + +// CheckStatus checks the status of a task on a specific queue and worker. +func (f *FastTaskService) CheckStatus(ctx context.Context, taskID, queueID, workerID string) (core.Phase, string, error) { + taskStatusChannelResult, exists := f.taskStatusChannels.Load(taskID) + if !exists { + // if this plugin restarts then TaskContexts may not exist for tasks that are still active. we can + // create a TaskContext here because we ensure it will be cleaned up when the task completes. + f.taskStatusChannels.Store(taskID, make(chan *workerTaskStatus, GetConfig().TaskStatusBufferSize)) + return core.PhaseUndefined, "", errors.New("task context not found") + } + + taskStatusChannel := taskStatusChannelResult.(chan *workerTaskStatus) + + var latestWorkerTaskStatus *workerTaskStatus +Loop: + for { + select { + case x := <-taskStatusChannel: + // ensure we retrieve the latest status from the worker that is currently assigned to the task + if x.workerID == workerID { + latestWorkerTaskStatus = x + } + default: + break Loop; + } + } + + if latestWorkerTaskStatus == nil { + return core.PhaseUndefined, "", fmt.Errorf("unable to find task status update: %w", statusUpdateNotFoundError) + } + + taskStatus := latestWorkerTaskStatus.taskStatus + phase := core.Phase(taskStatus.Phase) + + // if not completed need to send ACK on taskID to worker + if phase != core.PhaseSuccess && phase != core.PhaseRetryableFailure { + f.queuesLock.RLock() + defer f.queuesLock.RUnlock() + + queue, _ := f.queues[queueID] + queue.lock.RLock() + defer queue.lock.RUnlock() + + if worker, exists := queue.workers[workerID]; exists { + worker.responseChan <- &pb.HeartbeatResponse{ + TaskId: taskID, + Operation: pb.HeartbeatResponse_ACK, + } + } + } + + return phase, taskStatus.Reason, nil +} + +// Cleanup is used to indicate a task is no longer being tracked by the worker and delete the +// associated task context. +func (f *FastTaskService) Cleanup(ctx context.Context, taskID, queueID, workerID string) error { + // send delete taskID message to worker + f.queuesLock.RLock() + defer f.queuesLock.RUnlock() + + queue, _ := f.queues[queueID] + queue.lock.RLock() + defer queue.lock.RUnlock() + + if worker, exists := queue.workers[workerID]; exists { + worker.responseChan <- &pb.HeartbeatResponse{ + TaskId: taskID, + Operation: pb.HeartbeatResponse_DELETE, + } + } + + // delete task context + f.taskStatusChannels.Delete(taskID) + return nil +} + +// NewFastTaskService creates a new FastTaskService. +func NewFastTaskService(enqueueOwner core.EnqueueOwner) *FastTaskService { + return &FastTaskService{ + enqueueOwner: enqueueOwner, + queues: make(map[string]*Queue), + } +} diff --git a/fasttask/plugin/utils.go b/fasttask/plugin/utils.go new file mode 100644 index 0000000000..a4a9d170b1 --- /dev/null +++ b/fasttask/plugin/utils.go @@ -0,0 +1,32 @@ +package plugin + +import ( + "encoding/json" + "fmt" + + "k8s.io/api/core/v1" + + "github.com/unionai/flyte/fasttask/plugin/pb" +) + +// isValidEnvironmentSpec validates the FastTaskEnvironmentSpec +func isValidEnvironmentSpec(fastTaskEnvironmentSpec *pb.FastTaskEnvironmentSpec) error { + if fastTaskEnvironmentSpec.GetBacklogLength() < 0 { + return fmt.Errorf("backlog length must be greater than or equal to 0") + } + + if fastTaskEnvironmentSpec.GetParallelism() <= 0 { + return fmt.Errorf("parallelism must be greater than 0") + } + + podTemplateSpec := &v1.PodTemplateSpec{} + if err := json.Unmarshal(fastTaskEnvironmentSpec.PodTemplateSpec, podTemplateSpec); err != nil { + return fmt.Errorf("unable to unmarshal PodTemplateSpec [%v], Err: [%v]", fastTaskEnvironmentSpec.PodTemplateSpec, err.Error()) + } + + if fastTaskEnvironmentSpec.GetReplicaCount() <= 0 { + return fmt.Errorf("replica count must be greater than 0") + } + + return nil +} diff --git a/fasttask/protos/fasttask.proto b/fasttask/protos/fasttask.proto new file mode 100644 index 0000000000..c39a3dbd1d --- /dev/null +++ b/fasttask/protos/fasttask.proto @@ -0,0 +1,63 @@ +syntax = "proto3"; + +package fasttask; + +option go_package = "pb/"; + +import "google/protobuf/struct.proto"; + +message TaskStatus { + string taskId = 1; + string namespace = 2; + string workflowId = 3; + int32 phase = 4; + string reason = 5; +} + +message Capacity { + int32 executionCount = 1; + int32 executionLimit = 2; + int32 backlogCount = 3; + int32 backlogLimit = 4; +} + +message HeartbeatRequest { + string workerId = 1; + string queueId = 2; + Capacity capacity = 3; + repeated TaskStatus taskStatuses = 4; +} + +message HeartbeatResponse { + string taskId = 1; + string namespace = 2; + string workflowId = 3; + repeated string cmd = 4; + + enum Operation { + ACK = 0; + ASSIGN = 1; + DELETE = 2; + } + + Operation operation = 5; +} + +service FastTask { + rpc Heartbeat(stream HeartbeatRequest) returns (stream HeartbeatResponse); +} + +message FastTaskEnvironment { + string queue_id = 1; +} + +message FastTaskEnvironmentSpec { + int32 backlog_length = 1; + int32 parallelism = 2; + bytes pod_template_spec = 3; + string primary_container_name = 4; + int32 replica_count = 5; + oneof termination_criteria { + int32 ttl_seconds = 6; + } +} diff --git a/fasttask/worker/Cargo.lock b/fasttask/worker/Cargo.lock new file mode 100644 index 0000000000..78dc83aa69 --- /dev/null +++ b/fasttask/worker/Cargo.lock @@ -0,0 +1,1458 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "aho-corasick" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" +dependencies = [ + "memchr", +] + +[[package]] +name = "anstream" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ab91ebe16eb252986481c5b62f6098f3b698a45e34b5b98200cf20dd2484a44" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" + +[[package]] +name = "anstyle-parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "317b9a89c1868f5ea6ff1d9539a69f45dffc21ce321ac1fd1160dfa48c8e2140" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ca11d4be1bab0c8bc8734a9aa7bf4ee8316d462a08c6ac5052f888fef5b494b" +dependencies = [ + "windows-sys", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0699d10d2f4d628a98ee7b57b289abbc98ff3bad977cb3152709d4bf2330628" +dependencies = [ + "anstyle", + "windows-sys", +] + +[[package]] +name = "anyhow" +version = "1.0.75" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" + +[[package]] +name = "async-channel" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ca33f4bc4ed1babef42cad36cc1f51fa88be00420404e5b1e80ab1b18f7678c" +dependencies = [ + "concurrent-queue", + "event-listener", + "event-listener-strategy", + "futures-core", + "pin-project-lite", +] + +[[package]] +name = "async-stream" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd56dd203fef61ac097dd65721a419ddccb106b2d2b70ba60a6b529f03961a51" +dependencies = [ + "async-stream-impl", + "futures-core", + "pin-project-lite", +] + +[[package]] +name = "async-stream-impl" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "async-trait" +version = "0.1.73" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "autocfg" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" + +[[package]] +name = "axum" +version = "0.6.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" +dependencies = [ + "async-trait", + "axum-core", + "bitflags", + "bytes", + "futures-util", + "http", + "http-body", + "hyper", + "itoa", + "matchit", + "memchr", + "mime", + "percent-encoding", + "pin-project-lite", + "rustversion", + "serde", + "sync_wrapper", + "tower", + "tower-layer", + "tower-service", +] + +[[package]] +name = "axum-core" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "759fa577a247914fd3f7f76d62972792636412fbfd634cd452f6a385a74d2d2c" +dependencies = [ + "async-trait", + "bytes", + "futures-util", + "http", + "http-body", + "mime", + "rustversion", + "tower-layer", + "tower-service", +] + +[[package]] +name = "backtrace" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "base64" +version = "0.21.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "604178f6c5c21f02dc555784810edfb88d34ac2c73b2eae109655649ee73ce3d" + +[[package]] +name = "bincode" +version = "1.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1f45e9417d87227c7a56d22e471c6206462cba514c7590c09aff4cf6d1ddcad" +dependencies = [ + "serde", +] + +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bytes" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89b2fd2a0dcf38d7971e2194b6b6eebab45ae01067456a7fd93d5547a61b70be" + +[[package]] +name = "cc" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +dependencies = [ + "libc", +] + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "clap" +version = "4.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d04704f56c2cde07f43e8e2c154b43f216dc5c92fc98ada720177362f953b956" +dependencies = [ + "clap_builder", + "clap_derive", +] + +[[package]] +name = "clap_builder" +version = "4.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e231faeaca65ebd1ea3c737966bf858971cd38c3849107aa3ea7de90a804e45" +dependencies = [ + "anstream", + "anstyle", + "clap_lex", + "strsim", +] + +[[package]] +name = "clap_derive" +version = "4.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0862016ff20d69b84ef8247369fabf5c008a7417002411897d40ee1f4532b873" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "clap_lex" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd7cc57abe963c6d3b9d8be5b06ba7c8957a930305ca90304f24ef040aa6f961" + +[[package]] +name = "colorchoice" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" + +[[package]] +name = "concurrent-queue" +version = "2.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d16048cd947b08fa32c24458a22f5dc5e835264f689f4f5653210c69fd107363" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a22b2d63d4d1dc0b7f1b6b2747dd0088008a9be28b6ddf0b1e7d335e3037294" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "either" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" + +[[package]] +name = "event-listener" +version = "4.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "770d968249b5d99410d61f5bf89057f3199a077a04d087092f58e7d10692baae" +dependencies = [ + "concurrent-queue", + "parking", + "pin-project-lite", +] + +[[package]] +name = "event-listener-strategy" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "958e4d70b6d5e81971bebec42271ec641e7ff4e170a6fa605f2b8a8b65cb97d3" +dependencies = [ + "event-listener", + "pin-project-lite", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "futures" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" + +[[package]] +name = "futures-executor" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" + +[[package]] +name = "futures-macro" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "futures-sink" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" + +[[package]] +name = "futures-task" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" + +[[package]] +name = "futures-util" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "getrandom" +version = "0.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.28.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fb8d784f27acf97159b40fc4db5ecd8aa23b9ad5ef69cdd136d3bc80665f0c0" + +[[package]] +name = "h2" +version = "0.3.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" +dependencies = [ + "bytes", + "fnv", + "futures-core", + "futures-sink", + "futures-util", + "http", + "indexmap", + "slab", + "tokio", + "tokio-util", + "tracing", +] + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "hermit-abi" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" + +[[package]] +name = "http" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd6effc99afb63425aff9b05836f029929e345a6148a14b7ecd5ab67af944482" +dependencies = [ + "bytes", + "fnv", + "itoa", +] + +[[package]] +name = "http-body" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5f38f16d184e36f2408a55281cd658ecbd3ca05cce6d6510a176eca393e26d1" +dependencies = [ + "bytes", + "http", + "pin-project-lite", +] + +[[package]] +name = "httparse" +version = "1.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" + +[[package]] +name = "httpdate" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" + +[[package]] +name = "hyper" +version = "0.14.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffb1cfd654a8219eaef89881fdb3bb3b1cdc5fa75ded05d6933b2b382e395468" +dependencies = [ + "bytes", + "futures-channel", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "httparse", + "httpdate", + "itoa", + "pin-project-lite", + "socket2 0.4.9", + "tokio", + "tower-service", + "tracing", + "want", +] + +[[package]] +name = "hyper-timeout" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbb958482e8c7be4bc3cf272a766a2b0bf1a6755e7a6ae777f017a31d11b13b1" +dependencies = [ + "hyper", + "pin-project-lite", + "tokio", + "tokio-io-timeout", +] + +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown", +] + +[[package]] +name = "indoc" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e186cfbae8084e513daff4240b4797e342f988cecda4fb6c939150f96315fd8" + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" + +[[package]] +name = "libc" +version = "0.2.147" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" + +[[package]] +name = "lock_api" +version = "0.4.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" + +[[package]] +name = "matchers" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" +dependencies = [ + "regex-automata 0.1.10", +] + +[[package]] +name = "matchit" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed1202b2a6f884ae56f04cff409ab315c5ce26b5e58d7412e484f01fd52f52ef" + +[[package]] +name = "memchr" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" + +[[package]] +name = "memoffset" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a634b1c61a95585bd15607c6ab0c4e5b226e695ff2800ba0cdccddf208c406c" +dependencies = [ + "autocfg", +] + +[[package]] +name = "mime" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "mio" +version = "0.8.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2" +dependencies = [ + "libc", + "wasi", + "windows-sys", +] + +[[package]] +name = "nu-ansi-term" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +dependencies = [ + "overload", + "winapi", +] + +[[package]] +name = "num_cpus" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" +dependencies = [ + "hermit-abi", + "libc", +] + +[[package]] +name = "object" +version = "0.32.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77ac5bbd07aea88c60a577a1ce218075ffd59208b2d7ca97adf9bfc5aeb21ebe" +dependencies = [ + "memchr", +] + +[[package]] +name = "once_cell" +version = "1.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" + +[[package]] +name = "overload" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" + +[[package]] +name = "parking" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae" + +[[package]] +name = "parking_lot" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-targets", +] + +[[package]] +name = "percent-encoding" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94" + +[[package]] +name = "pin-project" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fda4ed1c6c173e3fc7a83629421152e01d7b1f9b7f65fb301e490e8cfc656422" +dependencies = [ + "pin-project-internal", +] + +[[package]] +name = "pin-project-internal" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12cc1b0bf1727a77a54b6654e7b5f1af8604923edc8b81885f8ec92f9e3f0a05" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "ppv-lite86" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" + +[[package]] +name = "proc-macro2" +version = "1.0.66" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "18fb31db3f9bddb2ea821cde30a9f70117e3f119938b5ee630b7403aa6e2ead9" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b82eaa1d779e9a4bc1c3217db8ffbeabaae1dca241bf70183242128d48681cd" +dependencies = [ + "bytes", + "prost-derive", +] + +[[package]] +name = "prost-derive" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5d2d8d10f3c6ded6da8b05b5fb3b8a5082514344d56c9f871412d29b4e075b4" +dependencies = [ + "anyhow", + "itertools", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "prost-types" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "213622a1460818959ac1181aaeb2dc9c7f63df720db7d788b3e24eacd1983e13" +dependencies = [ + "prost", +] + +[[package]] +name = "pyo3" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a89dc7a5850d0e983be1ec2a463a171d20990487c3cfcd68b5363f1ee3d6fe0" +dependencies = [ + "cfg-if", + "indoc", + "libc", + "memoffset", + "parking_lot", + "pyo3-build-config", + "pyo3-ffi", + "pyo3-macros", + "unindent", +] + +[[package]] +name = "pyo3-build-config" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07426f0d8fe5a601f26293f300afd1a7b1ed5e78b2a705870c5f30893c5163be" +dependencies = [ + "once_cell", + "target-lexicon", +] + +[[package]] +name = "pyo3-ffi" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbb7dec17e17766b46bca4f1a4215a85006b4c2ecde122076c562dd058da6cf1" +dependencies = [ + "libc", + "pyo3-build-config", +] + +[[package]] +name = "pyo3-macros" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f738b4e40d50b5711957f142878cfa0f28e054aa0ebdfc3fd137a843f74ed3" +dependencies = [ + "proc-macro2", + "pyo3-macros-backend", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "pyo3-macros-backend" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fc910d4851847827daf9d6cdd4a823fbdaab5b8818325c5e97a86da79e8881f" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "quote" +version = "1.0.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5267fca4496028628a95160fc423a33e8b2e6af8a5302579e322e4b520293cae" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "redox_syscall" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +dependencies = [ + "bitflags", +] + +[[package]] +name = "regex" +version = "1.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12de2eff854e5fa4b1295edd650e227e9d8fb0c9e90b12e7f36d6a6811791a29" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata 0.3.7", + "regex-syntax 0.7.5", +] + +[[package]] +name = "regex-automata" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49530408a136e16e5b486e883fbb6ba058e8e4e8ae6621a77b048b314336e629" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax 0.7.5", +] + +[[package]] +name = "regex-syntax" +version = "0.6.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" + +[[package]] +name = "regex-syntax" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" + +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustversion" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "serde" +version = "1.0.186" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f5db24220c009de9bd45e69fb2938f4b6d2df856aa9304ce377b3180f83b7c1" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.186" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ad697f7e0b65af4983a4ce8f56ed5b357e8d3c36651bf6a7e13639c17b8e670" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "sharded-slab" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f40ca3c46823713e0d4209592e8d6e826aa57e928f09752619fc696c499637f6" +dependencies = [ + "lazy_static", +] + +[[package]] +name = "signal-hook-registry" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1" +dependencies = [ + "libc", +] + +[[package]] +name = "slab" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] + +[[package]] +name = "smallvec" +version = "1.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6ecd384b10a64542d77071bd64bd7b231f4ed5940fba55e98c3de13824cf3d7" + +[[package]] +name = "socket2" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "socket2" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2538b18701741680e0322a2302176d3253a35388e2e62f172f64f4f16605f877" +dependencies = [ + "libc", + "windows-sys", +] + +[[package]] +name = "strsim" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c324c494eba9d92503e6f1ef2e6df781e78f6a7705a0202d9801b198807d518a" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "sync_wrapper" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" + +[[package]] +name = "target-lexicon" +version = "0.12.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69758bda2e78f098e4ccb393021a0963bb3442eac05f135c30f61b7370bbafae" + +[[package]] +name = "thread_local" +version = "1.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b9ef9bad013ada3808854ceac7b46812a6465ba368859a37e2100283d2d719c" +dependencies = [ + "cfg-if", + "once_cell", +] + +[[package]] +name = "tokio" +version = "1.32.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "17ed6077ed6cd6c74735e21f37eb16dc3935f96878b1fe961074089cc80893f9" +dependencies = [ + "backtrace", + "bytes", + "libc", + "mio", + "num_cpus", + "pin-project-lite", + "signal-hook-registry", + "socket2 0.5.3", + "tokio-macros", + "windows-sys", +] + +[[package]] +name = "tokio-io-timeout" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30b74022ada614a1b4834de765f9bb43877f910cc8ce4be40e89042c9223a8bf" +dependencies = [ + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tokio-macros" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "tokio-stream" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tokio-util" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "806fe8c2c87eccc8b3267cbae29ed3ab2d0bd37fca70ab622e46aaa9375ddb7d" +dependencies = [ + "bytes", + "futures-core", + "futures-sink", + "pin-project-lite", + "tokio", + "tracing", +] + +[[package]] +name = "tonic" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3082666a3a6433f7f511c7192923fa1fe07c69332d3c6a2e6bb040b569199d5a" +dependencies = [ + "async-trait", + "axum", + "base64", + "bytes", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "hyper", + "hyper-timeout", + "percent-encoding", + "pin-project", + "prost", + "tokio", + "tokio-stream", + "tower", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower" +version = "0.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" +dependencies = [ + "futures-core", + "futures-util", + "indexmap", + "pin-project", + "pin-project-lite", + "rand", + "slab", + "tokio", + "tokio-util", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower-layer" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c20c8dbed6283a09604c3e69b4b7eeb54e298b8a600d4d5ecb5ad39de609f1d0" + +[[package]] +name = "tower-service" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" + +[[package]] +name = "tracing" +version = "0.1.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +dependencies = [ + "cfg-if", + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.29", +] + +[[package]] +name = "tracing-core" +version = "0.1.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" +dependencies = [ + "once_cell", + "valuable", +] + +[[package]] +name = "tracing-log" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee855f1f400bd0e5c02d150ae5de3840039a3f54b025156404e34c23c03f47c3" +dependencies = [ + "log", + "once_cell", + "tracing-core", +] + +[[package]] +name = "tracing-subscriber" +version = "0.3.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad0f048c97dbd9faa9b7df56362b8ebcaa52adb06b498c050d2f4e32f90a7a8b" +dependencies = [ + "matchers", + "nu-ansi-term", + "once_cell", + "regex", + "sharded-slab", + "smallvec", + "thread_local", + "tracing", + "tracing-core", + "tracing-log", +] + +[[package]] +name = "try-lock" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" + +[[package]] +name = "unicode-ident" +version = "1.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "301abaae475aa91687eb82514b328ab47a211a533026cb25fc3e519b86adfc3c" + +[[package]] +name = "unindent" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7de7d73e1754487cb58364ee906a499937a0dfabd86bcb980fa99ec8c8fa2ce" + +[[package]] +name = "utf8parse" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" + +[[package]] +name = "uuid" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79daa5ed5740825c40b389c5e50312b9c86df53fccd33f281df655642b43869d" +dependencies = [ + "getrandom", +] + +[[package]] +name = "valuable" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" + +[[package]] +name = "want" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa7760aed19e106de2c7c0b581b509f2f25d3dacaf737cb82ac61bc6d760b0e" +dependencies = [ + "try-lock", +] + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-sys" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" +dependencies = [ + "windows-targets", +] + +[[package]] +name = "windows-targets" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" + +[[package]] +name = "windows_i686_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" + +[[package]] +name = "windows_i686_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" + +[[package]] +name = "worker" +version = "0.1.0" +dependencies = [ + "async-channel", + "async-stream", + "bincode", + "clap", + "futures", + "prost", + "prost-types", + "pyo3", + "serde", + "tokio", + "tokio-util", + "tonic", + "tracing", + "tracing-subscriber", + "uuid", +] diff --git a/fasttask/worker/Cargo.toml b/fasttask/worker/Cargo.toml new file mode 100644 index 0000000000..15c40777e2 --- /dev/null +++ b/fasttask/worker/Cargo.toml @@ -0,0 +1,28 @@ +[package] +name = "worker" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +async-channel = "2.1" +async-stream = "0.3" +bincode = "1.3" +clap = { version = "4.4.6", features = ["derive"] } +futures = "0.3" +tonic = "0.9" +prost = "0.11" +prost-types = "0.11" +pyo3 = "0.20.2" +tokio = { version = "1.0", features = ["macros", "net", "process", "rt", "rt-multi-thread"] } +tokio-util = { version = "0.7", features = ["codec"] } +tracing = "0.1" +tracing-subscriber = { version = "0.3", features = ["env-filter"] } +serde = { version = "1.0", features = ["derive"] } +uuid = { version = "1.4.1", features = ["v4"] } + +[profile.release] +strip = true +opt-level = "z" +lto = true diff --git a/fasttask/worker/Dockerfile b/fasttask/worker/Dockerfile new file mode 100644 index 0000000000..5cc9f140c3 --- /dev/null +++ b/fasttask/worker/Dockerfile @@ -0,0 +1,8 @@ +FROM rust:1.76-bookworm as builder +WORKDIR /usr/src/worker +COPY . . +RUN apt update && apt install -y python3-dev && cargo install --path . + +FROM debian:bookworm-slim +COPY --from=builder /usr/local/cargo/bin/worker /usr/local/bin/worker +CMD ["worker"] diff --git a/fasttask/worker/src/bridge.rs b/fasttask/worker/src/bridge.rs new file mode 100644 index 0000000000..0fa8307d45 --- /dev/null +++ b/fasttask/worker/src/bridge.rs @@ -0,0 +1,311 @@ +use std::collections::{HashMap, HashSet}; +use std::future::Future; +use std::sync::{Arc, Mutex, RwLock}; +use std::pin::Pin; +use std::task::{Context, Poll, Waker}; +use std::time::{UNIX_EPOCH, Duration, SystemTime}; + +use crate::{FAILED, SUCCEEDED, task, BridgeArgs, TaskContext}; +use crate::pb::fasttask::{Capacity, HeartbeatRequest, TaskStatus}; +use crate::pb::fasttask::heartbeat_response::Operation; +use crate::pb::fasttask::fast_task_client::FastTaskClient; +use crate::executor::Executor; + +use async_channel; +use tokio; +use tokio::net::TcpListener; +use tokio::time::Interval; +use tokio::process::Command; +use tokio_util::codec::{Framed, LengthDelimitedCodec}; +use tonic::Request; +use tracing::{debug, error, warn}; +use uuid::Uuid; + +struct AsyncBoolFuture { + async_bool: Arc>, +} + +impl Future for AsyncBoolFuture { + type Output = (); + + fn poll(self: Pin<&mut Self>, ctx: &mut Context<'_>) -> Poll<()> { + let mut async_bool = self.async_bool.lock().unwrap(); + if async_bool.value { + async_bool.value = false; + return Poll::Ready(()); + } + + let waker = ctx.waker().clone(); + async_bool.waker = Some(waker); + + Poll::Pending + } +} + +struct AsyncBool { + value: bool, + waker: Option, +} + +impl AsyncBool { + fn new() -> Self { + Self { + value: false, + waker: None, + } + } + + fn trigger(&mut self) { + self.value = true; + if let Some(waker) = &self.waker { + waker.clone().wake(); + } + } +} + +struct Heartbeater { + interval: Interval, + async_bool: Arc>, +} + +impl Heartbeater { + async fn trigger(&mut self) -> () { + let async_bool_future = AsyncBoolFuture { + async_bool: self.async_bool.clone(), + }; + + tokio::select! { + _ = self.interval.tick() => {}, + _ = async_bool_future => {}, + } + } +} + +pub async fn run(args: BridgeArgs, executor_registration_addr: &str) -> Result<(), Box> { + let worker_id = Uuid::new_v4().to_string(); // generate a random worker_id so that it is different on restart + let (task_status_tx, task_status_rx) = async_channel::unbounded(); + let task_statuses: Arc>> = Arc::new(RwLock::new(vec!())); + let heartbeat_bool = Arc::new(Mutex::new(AsyncBool::new())); + + let (backlog_tx, backlog_rx) = match args.backlog_length{ + 0 => (None, None), + x => { + let (tx, rx) = async_channel::bounded(x); + (Some(tx), Some(rx)) + } + }; + + // build executors + let (build_executor_tx, build_executor_rx) = async_channel::unbounded(); + let (executor_tx, executor_rx) = async_channel::unbounded(); + + let executor_tx_clone = executor_tx.clone(); + let listener = match TcpListener::bind(executor_registration_addr).await { + Ok(listener) => listener, + Err(e) => { + error!("failed to bind to port '{}'", e); + std::process::exit(1); + } + }; + + let executor_registration_addr = executor_registration_addr.to_string(); + tokio::spawn(async move { + let mut index = 0; + loop { + build_executor_rx.recv().await; + + // start child process + let child = Command::new(std::env::args().next().unwrap()) + .arg("--executor-registration-addr") + .arg(executor_registration_addr.clone()) + .arg("executor") + .arg("--id") + .arg(index.to_string()) + .spawn() + .unwrap(); + + let stream = listener.accept().await.unwrap().0; + let framed = Framed::new(stream, LengthDelimitedCodec::new()); + + let executor = Executor{ + framed, + child, + }; + executor_tx_clone.send(executor).await; + + index += 1; + } + }); + + for _ in 0..args.parallelism { + build_executor_tx.send(()).await; + } + + // spawn task status aggregator + let (heartbeat_bool_clone, task_statuses_clone) = (heartbeat_bool.clone(), task_statuses.clone()); + tokio::spawn(async move { + loop { + let task_status_result = task_status_rx.recv().await; + + // append task_status to task_statuses + let task_status: TaskStatus = task_status_result.unwrap(); + if task_status.phase == SUCCEEDED || task_status.phase == FAILED { + // if task phase is terminal then trigger heartbeat immediately + let mut heartbeat_bool = heartbeat_bool_clone.lock().unwrap(); + heartbeat_bool.trigger(); + } + let mut task_statuses = task_statuses_clone.write().unwrap(); + task_statuses.push(task_status); + } + }); + + let mut fast_register_ids = HashSet::new(); + loop { + // initialize grpc client + let mut client = match FastTaskClient::connect(args.fasttask_url.clone()).await { + Ok(client) => client, + Err(e) => { + error!("failed to connect to grpc service '{}' '{:?}'", args.fasttask_url, e); + tokio::time::sleep(Duration::from_secs(1)).await; + continue; + }, + }; + + // start heartbeater + let (worker_id_clone, queue_id_clone, task_statuses_clone, heartbeat_bool_clone, heartbeat_interval_seconds, fast_register_dir_override) = + (worker_id.clone(), args.queue_id.clone(), task_statuses.clone(), heartbeat_bool.clone(), args.heartbeat_interval_seconds, args.fast_register_dir_override.clone()); + let (executor_rx_clone, parallelism_clone, backlog_rx_clone, backlog_length_clone) = + (executor_rx.clone(), args.parallelism as i32, backlog_rx.clone(), args.backlog_length as i32); + let outbound = async_stream::stream! { + let mut heartbeater = Heartbeater { + interval: tokio::time::interval(Duration::from_secs(heartbeat_interval_seconds)), + async_bool: heartbeat_bool_clone.clone(), + }; + + loop { + // periodically send heartbeat + let _ = heartbeater.trigger().await; + + let backlogged = match backlog_rx_clone { + Some(ref rx) => rx.len() as i32, + None => 0, + }; + let mut heartbeat_request = HeartbeatRequest { + worker_id: worker_id_clone.clone(), + queue_id: queue_id_clone.clone(), + capacity: Some(Capacity { + execution_count: parallelism_clone - (executor_rx_clone.len() as i32), + execution_limit: parallelism_clone, + backlog_count: backlogged, + backlog_limit: backlog_length_clone, + }), + task_statuses: vec!(), + }; + + { + let mut task_statuses = task_statuses_clone.write().unwrap(); + heartbeat_request.task_statuses = task_statuses.clone(); + task_statuses.clear(); + } + + debug!("sending heartbeat request '{:?}'", heartbeat_request); + yield heartbeat_request; + } + }; + + // handle heartbeat responses + let response = match client.heartbeat(Request::new(outbound)).await { + Ok(response) => response, + Err(e) => { + warn!("failed to send heartbeat '{:?}'", e); + continue; + }, + }; + + let mut inbound = response.into_inner(); + + let task_contexts = Arc::new(RwLock::new(HashMap::::new())); + //while let Some(heartbeat_response) = inbound.message().await? { + loop { + let heartbeat_response_result = inbound.message().await; + if let Err(e) = heartbeat_response_result { + warn!("failed to retrieve heartbeat response '{:?}'", e); + break; + } + let heartbeat_response_option = heartbeat_response_result.unwrap(); + if let None = heartbeat_response_option { + break; + } + let heartbeat_response = heartbeat_response_option.unwrap(); + debug!("sending heartbeat response = {:?}", heartbeat_response); + + match Operation::from_i32(heartbeat_response.operation) { + Some(Operation::Assign) => { + // parse and update command + let mut cmd_str = heartbeat_response.cmd.clone(); + let (mut fast_register_id_index, mut pyflyte_execute_index) = (None, None); + if cmd_str[0].eq("pyflyte-fast-execute") { + for i in 0..cmd_str.len() { + match cmd_str[i] { + ref x if x.eq("--dest-dir") => cmd_str[i+1] = fast_register_dir_override.clone(), + ref x if x.eq("--additional-distribution") => fast_register_id_index = Some(i+1), + ref x if x.eq("pyflyte-execute") => pyflyte_execute_index = Some(i), + _ => (), + } + } + } + + // if fast register file has already been processed we update to skip the + // download and decompression steps + let mut cmd_index = 0; + if let (Some(i), Some(j)) = (fast_register_id_index, pyflyte_execute_index) { + if fast_register_ids.contains(&cmd_str[i]) { + cmd_index = j; + } else { + fast_register_ids.insert(cmd_str[i].clone()); + } + } + + // copy cmd_str vec from cmd_index + let cmd = cmd_str[cmd_index..].to_vec(); + //let mut cmd = tokio::process::Command::new(&cmd_str[cmd_index]); + //cmd.args(&cmd_str[cmd_index+1..]); + + // execute command + let (task_id, namespace, workflow_id) = + (heartbeat_response.task_id.clone(), heartbeat_response.namespace.clone(), heartbeat_response.workflow_id.clone()); + let (task_contexts_clone, task_status_tx_clone, task_status_report_interval_seconds, last_ack_grace_period_seconds) = + (task_contexts.clone(), task_status_tx.clone(), args.task_status_report_interval_seconds, args.last_ack_grace_period_seconds); + let (backlog_tx_clone, backlog_rx_clone, executor_tx_clone, executor_rx_clone, build_executor_tx_clone) = + (backlog_tx.clone(), backlog_rx.clone(), executor_tx.clone(), executor_rx.clone(), build_executor_tx.clone()); + tokio::spawn(async move { + if let Err(e) = task::execute(task_contexts_clone, task_id, namespace, workflow_id, cmd, task_status_tx_clone, + task_status_report_interval_seconds, last_ack_grace_period_seconds, + backlog_tx_clone, backlog_rx_clone, executor_tx_clone, executor_rx_clone, build_executor_tx_clone).await { + warn!("failed to execute task '{:?}'", e); + } + }); + }, + Some(Operation::Ack) => { + let mut task_contexts = task_contexts.write().unwrap(); + + // update last ack timestamp + if let Some(ref mut task_context) = task_contexts.get_mut(&heartbeat_response.task_id) { + task_context.last_ack_timestamp = SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs(); + } + }, + Some(Operation::Delete) => { + let mut task_contexts = task_contexts.write().unwrap(); + + // send kill signal + if let Some(ref mut task_context) = task_contexts.get_mut(&heartbeat_response.task_id) { + if let Err(e) = task_context.kill_tx.send(()).await { + warn!("failed to kill task '{:?}'", e); + } + } + }, + None => warn!("unsupported heartbeat request operation '{:?}'", heartbeat_response.operation), + } + } + } +} diff --git a/fasttask/worker/src/executor.rs b/fasttask/worker/src/executor.rs new file mode 100644 index 0000000000..ace908420d --- /dev/null +++ b/fasttask/worker/src/executor.rs @@ -0,0 +1,98 @@ +use crate::{ExecutorArgs, FAILED, SUCCEEDED}; + +use futures::sink::SinkExt; +use futures::stream::StreamExt; +use pyo3::prelude::*; +use pyo3::exceptions::PySystemExit; +use tokio::net::TcpStream; +use tokio_util::codec::{Framed, LengthDelimitedCodec}; +use tracing::{debug, error, info}; +use serde::{Deserialize, Serialize}; + +pub struct Executor { + pub framed: Framed, + pub child: tokio::process::Child, +} + +#[derive(Deserialize, Serialize)] +pub struct Task { + pub cmd: Vec, +} + +#[derive(Deserialize, Serialize)] +pub struct Response { + pub phase: i32, + pub reason: Option, +} + +pub async fn run(args: ExecutorArgs, executor_registration_addr: &str) -> Result<(), Box> { + let stream = TcpStream::connect(executor_registration_addr).await?; + let mut framed = Framed::new(stream, LengthDelimitedCodec::new()); + + pyo3::prepare_freethreaded_python(); + Python::with_gil(|py| { + // import to reduce python environment initialization time + let _flytekit = PyModule::import(py, "flytekit").unwrap(); + let _entrypoint = PyModule::import(py, "flytekit.bin.entrypoint").unwrap(); + }); + + loop { + let buf = match framed.next().await { + Some(Ok(buf)) => buf, + Some(Err(e)) => { + error!("executor '{}' failed to read from socket: {}", args.id, e); + break; + }, + None => { + error!("executor '{}' connection closed", args.id); + break; + }, + }; + + let task: Task = bincode::deserialize(&buf).unwrap(); + debug!("executor {} received work: {:?}", args.id, task.cmd); + + let result = Python::with_gil(|py| { + let entrypoint = PyModule::import(py, "flytekit.bin.entrypoint").unwrap(); + + // convert cmd to &str + let cmd = task.cmd.iter().skip(1).map(|s| s.as_str()).collect::>(); + + // execute command + match task.cmd[0].as_str() { + "pyflyte-fast-execute" => { + match entrypoint.call_method1("fast_execute_task_cmd", (cmd,),) { + Ok(_) => Ok(()), + Err(e) if e.is_instance_of::(py) => Ok(()), + Err(e) => Err(format!("{:?}", e)), + } + }, + "pyflyte-execute" => { + match entrypoint.call_method1("execute_task_cmd", (cmd,),) { + Ok(_) => Ok(()), + Err(e) if e.is_instance_of::(py) => Ok(()), + Err(e) => Err(format!("{:?}", e)), + } + }, + _ => Err(format!("unsupported task command '{}'", task.cmd[0])), + } + }); + + let response = match result { + Ok(_) => Response { + phase: SUCCEEDED, + reason: None, + }, + Err(e) => Response { + phase: FAILED, + reason: Some(e), + }, + }; + + let buf = bincode::serialize(&response).unwrap(); + framed.send(buf.into()).await.unwrap(); + } + + info!("executor {} exiting", args.id); + Ok(()) +} diff --git a/fasttask/worker/src/main.rs b/fasttask/worker/src/main.rs new file mode 100644 index 0000000000..9f40219d55 --- /dev/null +++ b/fasttask/worker/src/main.rs @@ -0,0 +1,99 @@ +mod bridge; +mod pb; +mod task; +mod executor; + +use async_channel::{self, Sender}; +use clap::{Args, Parser, Subcommand}; +use tokio; +use tokio::runtime::Builder; +use tracing::{self, error}; +use tracing_subscriber::{self, EnvFilter}; + +const FAILED: i32 = 7; // 7=retryable 8=permanent +const QUEUED: i32 = 3; +const RUNNING: i32 = 5; +const SUCCEEDED: i32 = 6; + +pub struct TaskContext { + kill_tx: Sender<()>, + last_ack_timestamp: u64, +} + +#[derive(Debug, Parser)] +#[command(name = "worker", about = "fasttask worker", long_about = None)] +struct Cli { + #[arg(short, long, value_name = "EXECUTOR_REGISTRATION_ADDR", default_value = "127.0.0.1:15606", help = "endpoint for executor registration service")] + executor_registration_addr: String, + + #[command(subcommand)] + command: Option, +} + +#[derive(Debug, Subcommand)] +enum Commands { + Bridge(BridgeArgs), + Executor(ExecutorArgs), +} + +#[derive(Debug, Args)] +pub struct BridgeArgs { + #[arg(short, long, value_name = "FASTTASK_URL", default_value = "http://localhost:15605", help = "endpoint url for fasttask service")] + fasttask_url: String, + #[arg(short, long, value_name = "QUEUE_ID", default_value = "foo", help = "fasttask queue to listen on for tasks")] + queue_id: String, + #[arg(short = 'i', long, value_name = "HEARTBEAT_INTERVAL_SECONDS", default_value = "5", help = "interval in seconds to send heartbeat to fasttask service")] + heartbeat_interval_seconds: u64, + #[arg(short, long, value_name = "TASK_STATUS_REPORT_INTERVAL_SECONDS", default_value = "5", help = "interval in seconds to buffer task status for heartbeat")] + task_status_report_interval_seconds: u64, + #[arg(short, long, value_name = "LAST_ACK_GRACE_PERIOD_SECONDS", default_value = "10", help = "grace period in seconds to wait for last ack before killing task")] + last_ack_grace_period_seconds: u64, + #[arg(short, long, value_name = "PARALLELISM", default_value = "1", help = "number of tasks to run in parallel")] + parallelism: usize, + #[arg(short, long, value_name = "BACKLOG_LENGTH", default_value = "0", help = "number of tasks to buffer before dropping assignments")] + backlog_length: usize, + #[arg(short='r', long, value_name = "FAST_REGISTER_DIR_OVERRIDE", default_value = "/root", help = "directory to decompress flyte fast registration files")] + fast_register_dir_override: String, +} + +#[derive(Debug, Args)] +pub struct ExecutorArgs { + #[arg(short, long, value_name = "ID", help = "unique identifier for this executor instance")] + id: usize, +} + +fn main() -> Result<(), Box> { + let args = Cli::parse(); + let subscriber = tracing_subscriber::fmt() + .with_env_filter(EnvFilter::from_default_env()) + .finish(); + + tracing::subscriber::with_default(subscriber, || { + match args.command { + Some(Commands::Bridge(bridge_args)) => { + let runtime = Builder::new_current_thread() + .enable_all() + .build() + .unwrap(); + + if let Err(e) = runtime.block_on(bridge::run(bridge_args, &args.executor_registration_addr)) { + error!("failed to execute bridge: '{}'", e); + } + }, + Some(Commands::Executor(executor_args)) => { + + let runtime = Builder::new_current_thread() + .enable_io() + .build() + .unwrap(); + + if let Err(e) = runtime.block_on(executor::run(executor_args, &args.executor_registration_addr)) { + error!("failed to execute executor: '{}'", e); + } + }, + None => error!("unreachable"), + } + }); + + Ok(()) +} diff --git a/fasttask/worker/src/pb/fasttask.rs b/fasttask/worker/src/pb/fasttask.rs new file mode 100644 index 0000000000..db469dd2f4 --- /dev/null +++ b/fasttask/worker/src/pb/fasttask.rs @@ -0,0 +1,118 @@ +// @generated +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TaskStatus { + #[prost(string, tag="1")] + pub task_id: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub namespace: ::prost::alloc::string::String, + #[prost(string, tag="3")] + pub workflow_id: ::prost::alloc::string::String, + #[prost(int32, tag="4")] + pub phase: i32, + #[prost(string, tag="5")] + pub reason: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Capacity { + #[prost(int32, tag="1")] + pub execution_count: i32, + #[prost(int32, tag="2")] + pub execution_limit: i32, + #[prost(int32, tag="3")] + pub backlog_count: i32, + #[prost(int32, tag="4")] + pub backlog_limit: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct HeartbeatRequest { + #[prost(string, tag="1")] + pub worker_id: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub queue_id: ::prost::alloc::string::String, + #[prost(message, optional, tag="3")] + pub capacity: ::core::option::Option, + #[prost(message, repeated, tag="4")] + pub task_statuses: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct HeartbeatResponse { + #[prost(string, tag="1")] + pub task_id: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub namespace: ::prost::alloc::string::String, + #[prost(string, tag="3")] + pub workflow_id: ::prost::alloc::string::String, + #[prost(string, repeated, tag="4")] + pub cmd: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(enumeration="heartbeat_response::Operation", tag="5")] + pub operation: i32, +} +/// Nested message and enum types in `HeartbeatResponse`. +pub mod heartbeat_response { + #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] + #[repr(i32)] + pub enum Operation { + Ack = 0, + Assign = 1, + Delete = 2, + } + impl Operation { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + Operation::Ack => "ACK", + Operation::Assign => "ASSIGN", + Operation::Delete => "DELETE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "ACK" => Some(Self::Ack), + "ASSIGN" => Some(Self::Assign), + "DELETE" => Some(Self::Delete), + _ => None, + } + } + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FastTaskEnvironment { + #[prost(string, tag="1")] + pub queue_id: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FastTaskEnvironmentSpec { + #[prost(int32, tag="1")] + pub backlog_length: i32, + #[prost(int32, tag="2")] + pub parallelism: i32, + #[prost(bytes="vec", tag="3")] + pub pod_template_spec: ::prost::alloc::vec::Vec, + #[prost(string, tag="4")] + pub primary_container_name: ::prost::alloc::string::String, + #[prost(int32, tag="5")] + pub replica_count: i32, + #[prost(oneof="fast_task_environment_spec::TerminationCriteria", tags="6")] + pub termination_criteria: ::core::option::Option, +} +/// Nested message and enum types in `FastTaskEnvironmentSpec`. +pub mod fast_task_environment_spec { + #[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum TerminationCriteria { + #[prost(int32, tag="6")] + TtlSeconds(i32), + } +} +include!("fasttask.tonic.rs"); +// @@protoc_insertion_point(module) \ No newline at end of file diff --git a/fasttask/worker/src/pb/fasttask.tonic.rs b/fasttask/worker/src/pb/fasttask.tonic.rs new file mode 100644 index 0000000000..1a82dfe7ea --- /dev/null +++ b/fasttask/worker/src/pb/fasttask.tonic.rs @@ -0,0 +1,302 @@ +// @generated +/// Generated client implementations. +pub mod fast_task_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + /// + #[derive(Debug, Clone)] + pub struct FastTaskClient { + inner: tonic::client::Grpc, + } + impl FastTaskClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl FastTaskClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> FastTaskClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + FastTaskClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + /// + pub async fn heartbeat( + &mut self, + request: impl tonic::IntoStreamingRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/fasttask.FastTask/Heartbeat", + ); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert(GrpcMethod::new("fasttask.FastTask", "Heartbeat")); + self.inner.streaming(req, path, codec).await + } + } +} +/// Generated server implementations. +pub mod fast_task_server { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + /// Generated trait containing gRPC methods that should be implemented for use with FastTaskServer. + #[async_trait] + pub trait FastTask: Send + Sync + 'static { + /// Server streaming response type for the Heartbeat method. + type HeartbeatStream: futures_core::Stream< + Item = std::result::Result, + > + + Send + + 'static; + /// + async fn heartbeat( + &self, + request: tonic::Request>, + ) -> std::result::Result, tonic::Status>; + } + /// + #[derive(Debug)] + pub struct FastTaskServer { + inner: _Inner, + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, + } + struct _Inner(Arc); + impl FastTaskServer { + pub fn new(inner: T) -> Self { + Self::from_arc(Arc::new(inner)) + } + pub fn from_arc(inner: Arc) -> Self { + let inner = _Inner(inner); + Self { + inner, + accept_compression_encodings: Default::default(), + send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, + } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> InterceptedService + where + F: tonic::service::Interceptor, + { + InterceptedService::new(Self::new(inner), interceptor) + } + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } + } + impl tonic::codegen::Service> for FastTaskServer + where + T: FastTask, + B: Body + Send + 'static, + B::Error: Into + Send + 'static, + { + type Response = http::Response; + type Error = std::convert::Infallible; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(Ok(())) + } + fn call(&mut self, req: http::Request) -> Self::Future { + let inner = self.inner.clone(); + match req.uri().path() { + "/fasttask.FastTask/Heartbeat" => { + #[allow(non_camel_case_types)] + struct HeartbeatSvc(pub Arc); + impl< + T: FastTask, + > tonic::server::StreamingService + for HeartbeatSvc { + type Response = super::HeartbeatResponse; + type ResponseStream = T::HeartbeatStream; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request< + tonic::Streaming, + >, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { (*inner).heartbeat(request).await }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let inner = inner.0; + let method = HeartbeatSvc(inner); + let codec = tonic::codec::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.streaming(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + _ => { + Box::pin(async move { + Ok( + http::Response::builder() + .status(200) + .header("grpc-status", "12") + .header("content-type", "application/grpc") + .body(empty_body()) + .unwrap(), + ) + }) + } + } + } + } + impl Clone for FastTaskServer { + fn clone(&self) -> Self { + let inner = self.inner.clone(); + Self { + inner, + accept_compression_encodings: self.accept_compression_encodings, + send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, + } + } + } + impl Clone for _Inner { + fn clone(&self) -> Self { + Self(Arc::clone(&self.0)) + } + } + impl std::fmt::Debug for _Inner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{:?}", self.0) + } + } + impl tonic::server::NamedService for FastTaskServer { + const NAME: &'static str = "fasttask.FastTask"; + } +} diff --git a/fasttask/worker/src/pb/mod.rs b/fasttask/worker/src/pb/mod.rs new file mode 100644 index 0000000000..b12065b3aa --- /dev/null +++ b/fasttask/worker/src/pb/mod.rs @@ -0,0 +1 @@ +pub mod fasttask; diff --git a/fasttask/worker/src/task.rs b/fasttask/worker/src/task.rs new file mode 100644 index 0000000000..ba000ebe29 --- /dev/null +++ b/fasttask/worker/src/task.rs @@ -0,0 +1,319 @@ +use std::collections::HashMap; +use std::sync::{Arc, RwLock}; +use std::time::{UNIX_EPOCH, Instant, Duration, SystemTime}; + +use crate::{FAILED, QUEUED, RUNNING}; +use crate::TaskContext; +use crate::pb::fasttask::TaskStatus; +use crate::executor::{Task, Executor, Response}; + +use async_channel::{self, Receiver, Sender, TryRecvError, TrySendError}; +use futures::sink::SinkExt; +use futures::stream::StreamExt; +use tokio; +use tracing::{debug, info, warn}; + +pub async fn execute( + task_contexts: Arc>>, + task_id: String, + namespace: String, + workflow_id: String, + cmd: Vec, + task_status_tx: Sender, + task_status_report_interval_seconds: u64, + last_ack_grace_period_seconds: u64, + backlog_tx: Option>, + backlog_rx: Option>, + executor_tx: Sender, + executor_rx: Receiver, + build_executor_tx: Sender<()>, + ) -> Result<(), Box> { + + // check if task may be executed or backlogged + let (mut executor, backlogged) = is_executable(&executor_rx, &backlog_tx).await?; + + info!("starting task execution task_id={:?} executor={:?} backlogged={:?}", task_id, executor.is_some(), backlogged); + if executor.is_none() && !backlogged { + // if no executor and not backlogged then we drop the task transparently and allow grace + // period to failover to another worker + return Ok(()); + } + + // create and store new task context + let (kill_tx, kill_rx) = async_channel::bounded(1); + { + let mut task_contexts = task_contexts.write().unwrap(); + task_contexts.insert(task_id.clone(), TaskContext{ + kill_tx: kill_tx, + last_ack_timestamp: SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs(), + }); + } + + // if backlogged we wait until we can execute + let (mut phase, mut reason) = (QUEUED, "".to_string()); + if backlogged { + let backlog_rx = backlog_rx.unwrap(); + executor = match wait_in_backlog(task_contexts.clone(), &kill_rx, &task_id, &namespace, &workflow_id, &task_status_tx, + task_status_report_interval_seconds, last_ack_grace_period_seconds, &mut phase, &mut reason, &executor_rx, &backlog_rx).await { + Ok(executor) => executor, + Err(e) => return Err(e), + }; + } + + // execute task by running command - the only way that executor is None is if the task is + // previous killed during the previous `wait_in_backlog` function call + let killed = if let Some(mut executor) = executor { + let result = match run_command(task_contexts.clone(), &kill_rx, &task_id, &namespace, &workflow_id, cmd, &task_status_tx, + task_status_report_interval_seconds, last_ack_grace_period_seconds, &mut phase, &mut reason, &mut executor).await { + Ok(result) => Ok(result), + Err(e) => Err(format!("failed to run command: {:?}", e)), + }; + + // if the executor is dropped the child process is automatically killed, so we either + // re-enqueue the existing executor or indicate that a new instance should be created + match result { + Ok(true) => { + build_executor_tx.send(()).await?; + true + }, + Ok(false) => { + executor_tx.send(executor).await?; + false + }, + Err(e) => { + executor_tx.send(executor).await?; + return Err(e.into()); + }, + } + } else { + true + }; + + // if not closed then report terminal status + if !killed { + if let Err(e) = report_terminal_status(task_contexts.clone(), &kill_rx, &task_id, &namespace, &workflow_id, &task_status_tx, + task_status_report_interval_seconds, last_ack_grace_period_seconds, &mut phase, &mut reason).await { + return Err(e); + } + } + + // remove task context and open parallelism slot + { + let mut task_contexts = task_contexts.write().unwrap(); + task_contexts.remove(&task_id); + } + + Ok(()) +} + +async fn is_executable(executor_rx: &Receiver, backlog_tx: &Option>) -> Result<(Option, bool), String> { + match executor_rx.try_recv() { + Ok(executor) => return Ok((Some(executor), false)), + Err(TryRecvError::Closed) => return Err("executor_rx is closed".into()), + Err(TryRecvError::Empty) => {}, + } + + if let Some(backlog_tx) = backlog_tx { + match backlog_tx.try_send(()) { + Ok(_) => return Ok((None, true)), + Err(TrySendError::Closed(e)) => return Err(format!("backlog_tx is closed: {:?}", e)), + Err(TrySendError::Full(_)) => {}, + } + } + + Ok((None, false)) +} + +async fn report_terminal_status(task_contexts: Arc>>, kill_rx: &Receiver<()>, task_id: &str, namespace: &str, workflow_id: &str, + task_status_tx: &Sender, task_status_report_interval_seconds: u64, last_ack_grace_period_seconds: u64, phase: &mut i32, reason: &mut String) -> Result<(), Box> { + // send completed task status until deleted + let mut interval = tokio::time::interval(Duration::from_secs(task_status_report_interval_seconds)); + loop { + tokio::select!{ + _ = interval.tick() => { + // if last_ack_timestamp > grace_period then kill tasks and delete from task_contexts + let last_ack_timestamp; + { + let task_contexts = task_contexts.read().unwrap(); + + // can only be `Some` because this thread is the only way to delete + let task_context = task_contexts.get(task_id).unwrap(); + last_ack_timestamp = task_context.last_ack_timestamp; + } + + if SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs() - last_ack_timestamp > last_ack_grace_period_seconds { + warn!("task timed out task_id={:?}", task_id); + kill_rx.close(); + return Ok(()); + } + + // send task status + let error = task_status_tx.send(TaskStatus{ + task_id: task_id.to_string(), + namespace: namespace.to_string(), + workflow_id: workflow_id.to_string(), + phase: *phase, + reason: reason.clone(), + }).await; + + if error.is_err() { + warn!("failed to send task status error='{:?}'", error); + } + }, + _ = kill_rx.recv() => { + kill_rx.close(); + return Ok(()); + }, + } + } +} + +async fn run_command(task_contexts: Arc>>, kill_rx: &Receiver<()>, task_id: &str, namespace: &str, workflow_id: &str, + cmd: Vec, task_status_tx: &Sender, task_status_report_interval_seconds: u64, last_ack_grace_period_seconds: u64, phase: &mut i32, reason: &mut String, executor: &mut Executor) -> Result> { + // execute command and monitor + let task_start_ts = Instant::now(); + + let buf = bincode::serialize(&Task{ + cmd + }).unwrap(); + executor.framed.send(buf.into()).await.unwrap(); + + let mut interval = tokio::time::interval(Duration::from_secs(task_status_report_interval_seconds)); + loop { + tokio::select!{ + result = executor.framed.next() => { + info!("completed task_id {} in {}", task_id, task_start_ts.elapsed().as_millis()); + let buf = result.unwrap().unwrap(); + + let response: Response = bincode::deserialize(&buf).unwrap(); + + *phase = response.phase; + *reason = response.reason.unwrap_or("".to_string()); + break; + }, + result = executor.child.wait() => { + match result { + Ok(exit_status) => { + *phase = FAILED; + *reason = format!("process completed with exit status: '{}'", exit_status); + return Ok(true); + }, + Err(e) => { + *phase = FAILED; + *reason = format!("{}", e); + return Ok(true); + }, + } + }, + _ = interval.tick() => { + // if last_ack_timestamp > grace_period then kill tasks and delete from task_contexts + let last_ack_timestamp; + { + let task_contexts = task_contexts.read().unwrap(); + + // can only be `Some` because this thread is the only way to delete + let task_context = task_contexts.get(task_id).unwrap(); + last_ack_timestamp = task_context.last_ack_timestamp; + } + + if SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs() - last_ack_timestamp > last_ack_grace_period_seconds { + warn!("task_id = {:?} is timed out", task_id); + kill_rx.close(); + + *phase = FAILED; + *reason = "task timed out on last ack".to_string(); + + // to abort the process we attempt to kill, if this fails it will automatically + // be aborted when the instance is dropped. + let _ = executor.child.kill().await; + return Ok(true); + } + + // send task status + let error = task_status_tx.send(TaskStatus{ + task_id: task_id.to_string(), + namespace: namespace.to_string(), + workflow_id: workflow_id.to_string(), + phase: *phase, + reason: reason.clone(), + }).await; + + if error.is_err() { + warn!("ERROR = {:?}", error); + } + }, + _ = kill_rx.recv() => { + debug!("task was killed task_id={:?}", task_id); + kill_rx.close(); + + *phase = FAILED; + *reason = "task was killed".to_string(); + + // to abort the process we attempt to kill, if this fails it will automatically + // be aborted when the instance is dropped + let _ = executor.child.kill().await; + return Ok(true); + }, + } + } + + Ok(false) +} + +async fn wait_in_backlog(task_contexts: Arc>>, kill_rx: &Receiver<()>, task_id: &str, namespace: &str, workflow_id: &str, + task_status_tx: &Sender, task_status_report_interval_seconds: u64, last_ack_grace_period_seconds: u64, phase: &mut i32, reason: &mut String, + executor_rx: &Receiver, backlog_rx: &Receiver<()>) -> Result, Box> { + + let mut interval = tokio::time::interval(Duration::from_secs(task_status_report_interval_seconds)); + loop { + tokio::select!{ + result = executor_rx.recv() => { + let executor = match result { + Ok(executor) => executor, + Err(e) => return Err(format!("failed to retrieve executor: {:?}", e).into()), + }; + + if let Err(_) = backlog_rx.recv().await { + return Err("backlog_rx is closed".into()); + } + + *phase = RUNNING; + return Ok(Some(executor)); + }, + _ = interval.tick() => { + // if last_ack_timestamp > grace_period then kill tasks and delete from task_contexts + let last_ack_timestamp; + { + let task_contexts = task_contexts.read().unwrap(); + + // can only be `Some` because this thread is the only way to delete + let task_context = task_contexts.get(task_id).unwrap(); + last_ack_timestamp = task_context.last_ack_timestamp; + } + + if SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs() - last_ack_timestamp > last_ack_grace_period_seconds { + warn!("task timed out task_id={:?}", task_id); + kill_rx.close(); + return Ok(None); + } + + // send task status + let error = task_status_tx.send(TaskStatus{ + task_id: task_id.to_string(), + namespace: namespace.to_string(), + workflow_id: workflow_id.to_string(), + phase: *phase, + reason: reason.clone(), + }).await; + + if error.is_err() { + warn!("failed to send task status error='{:?}'", error); + } + }, + _ = kill_rx.recv() => { + kill_rx.close(); + return Ok(None); + }, + } + } +} diff --git a/flyteadmin/pkg/manager/impl/shared/iface.go b/flyteadmin/pkg/manager/impl/shared/iface.go index 94f5ced66c..1bbd315ac0 100644 --- a/flyteadmin/pkg/manager/impl/shared/iface.go +++ b/flyteadmin/pkg/manager/impl/shared/iface.go @@ -27,4 +27,6 @@ type WorkflowExecutionConfigInterface interface { GetOverwriteCache() bool // GetEnvs defines environment variables to be set for the execution. GetEnvs() *admin.Envs + // GetExecutionEnvAssignments defines execution environment assignments to be set for the execution. + GetExecutionEnvAssignments() []*core.ExecutionEnvAssignment } diff --git a/flyteadmin/pkg/manager/impl/util/shared.go b/flyteadmin/pkg/manager/impl/util/shared.go index 9c5b93e682..1a54903622 100644 --- a/flyteadmin/pkg/manager/impl/util/shared.go +++ b/flyteadmin/pkg/manager/impl/util/shared.go @@ -345,5 +345,10 @@ func MergeIntoExecConfig(workflowExecConfig admin.WorkflowExecutionConfig, spec workflowExecConfig.Envs = spec.GetEnvs() } + if (workflowExecConfig.GetExecutionEnvAssignments() == nil || len(workflowExecConfig.GetExecutionEnvAssignments()) == 0) && + (spec.GetExecutionEnvAssignments() != nil && len(spec.GetExecutionEnvAssignments()) > 0) { + workflowExecConfig.ExecutionEnvAssignments = spec.GetExecutionEnvAssignments() + } + return workflowExecConfig } diff --git a/flyteadmin/pkg/runtime/interfaces/application_configuration.go b/flyteadmin/pkg/runtime/interfaces/application_configuration.go index 3a459209f5..d06067931f 100644 --- a/flyteadmin/pkg/runtime/interfaces/application_configuration.go +++ b/flyteadmin/pkg/runtime/interfaces/application_configuration.go @@ -188,6 +188,12 @@ func (a *ApplicationConfig) GetEnvs() *admin.Envs { } } +// GetExecutionEnvAssignments is required to satisfy the ExecutionConfig interface. However, it +// does not make sense to expose this at the application level so we always return nil. +func (a *ApplicationConfig) GetExecutionEnvAssignments() []*core.ExecutionEnvAssignment { + return nil +} + // GetAsWorkflowExecutionConfig returns the WorkflowExecutionConfig as extracted from this object func (a *ApplicationConfig) GetAsWorkflowExecutionConfig() admin.WorkflowExecutionConfig { // These values should always be set as their fallback values equals to their zero value or nil, diff --git a/flyteadmin/pkg/workflowengine/impl/prepare_execution.go b/flyteadmin/pkg/workflowengine/impl/prepare_execution.go index 91642599a6..ca8bb783d2 100644 --- a/flyteadmin/pkg/workflowengine/impl/prepare_execution.go +++ b/flyteadmin/pkg/workflowengine/impl/prepare_execution.go @@ -73,6 +73,13 @@ func addExecutionOverrides(taskPluginOverrides []*admin.PluginOverride, } executionConfig.EnvironmentVariables = envs } + + for _, executionEnvAssignment := range workflowExecutionConfig.GetExecutionEnvAssignments() { + executionEnvAssignment := *executionEnvAssignment + executionConfig.ExecutionEnvAssignments = append(executionConfig.ExecutionEnvAssignments, v1alpha1.ExecutionEnvAssignment{ + ExecutionEnvAssignment: &executionEnvAssignment, + }) + } } if taskResources != nil { var requests = v1alpha1.TaskResourceSpec{} diff --git a/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts b/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts index ac9e059465..be37198aa2 100644 --- a/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts +++ b/flyteidl/gen/pb-es/flyteidl/admin/execution_pb.ts @@ -12,6 +12,7 @@ import { Annotations, AuthRole, Envs, Labels, Notification, RawOutputDataConfig, import { ArtifactID } from "../core/artifact_id_pb.js"; import { SecurityContext } from "../core/security_pb.js"; import { ClusterAssignment } from "./cluster_assignment_pb.js"; +import { ExecutionEnvAssignment } from "../core/execution_envs_pb.js"; import { Span } from "../core/metrics_pb.js"; /** @@ -1115,6 +1116,13 @@ export class ExecutionSpec extends Message { */ tags: string[] = []; + /** + * Execution environment assignments to be set for the execution. + * + * @generated from field: repeated flyteidl.core.ExecutionEnvAssignment execution_env_assignments = 25; + */ + executionEnvAssignments: ExecutionEnvAssignment[] = []; + constructor(data?: PartialMessage) { super(); proto3.util.initPartial(data, this); @@ -1140,6 +1148,7 @@ export class ExecutionSpec extends Message { { no: 22, name: "overwrite_cache", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, { no: 23, name: "envs", kind: "message", T: Envs }, { no: 24, name: "tags", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, + { no: 25, name: "execution_env_assignments", kind: "message", T: ExecutionEnvAssignment, repeated: true }, ]); static fromBinary(bytes: Uint8Array, options?: Partial): ExecutionSpec { diff --git a/flyteidl/gen/pb-es/flyteidl/admin/launch_plan_pb.ts b/flyteidl/gen/pb-es/flyteidl/admin/launch_plan_pb.ts index ee8c14bdcd..6e70237958 100644 --- a/flyteidl/gen/pb-es/flyteidl/admin/launch_plan_pb.ts +++ b/flyteidl/gen/pb-es/flyteidl/admin/launch_plan_pb.ts @@ -11,6 +11,7 @@ import { LiteralMap } from "../core/literals_pb.js"; import { Annotations, AuthRole, Envs, Labels, NamedEntityIdentifier, Notification, RawOutputDataConfig, Sort } from "./common_pb.js"; import { SecurityContext } from "../core/security_pb.js"; import { QualityOfService } from "../core/execution_pb.js"; +import { ExecutionEnvAssignment } from "../core/execution_envs_pb.js"; import { Schedule } from "./schedule_pb.js"; /** @@ -412,6 +413,13 @@ export class LaunchPlanSpec extends Message { */ envs?: Envs; + /** + * Execution environment assignments to be set for the execution. + * + * @generated from field: repeated flyteidl.core.ExecutionEnvAssignment execution_env_assignments = 22; + */ + executionEnvAssignments: ExecutionEnvAssignment[] = []; + constructor(data?: PartialMessage) { super(); proto3.util.initPartial(data, this); @@ -436,6 +444,7 @@ export class LaunchPlanSpec extends Message { { no: 19, name: "interruptible", kind: "message", T: BoolValue }, { no: 20, name: "overwrite_cache", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, { no: 21, name: "envs", kind: "message", T: Envs }, + { no: 22, name: "execution_env_assignments", kind: "message", T: ExecutionEnvAssignment, repeated: true }, ]); static fromBinary(bytes: Uint8Array, options?: Partial): LaunchPlanSpec { diff --git a/flyteidl/gen/pb-es/flyteidl/admin/matchable_resource_pb.ts b/flyteidl/gen/pb-es/flyteidl/admin/matchable_resource_pb.ts index 421358694a..d39ea6bf73 100644 --- a/flyteidl/gen/pb-es/flyteidl/admin/matchable_resource_pb.ts +++ b/flyteidl/gen/pb-es/flyteidl/admin/matchable_resource_pb.ts @@ -7,6 +7,7 @@ import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialM import { BoolValue, Message, proto3 } from "@bufbuild/protobuf"; import { SecurityContext } from "../core/security_pb.js"; import { Annotations, Envs, Labels, RawOutputDataConfig } from "./common_pb.js"; +import { ExecutionEnvAssignment } from "../core/execution_envs_pb.js"; import { QualityOfService } from "../core/execution_pb.js"; import { ClusterAssignment } from "./cluster_assignment_pb.js"; @@ -499,6 +500,13 @@ export class WorkflowExecutionConfig extends Message { */ envs?: Envs; + /** + * Execution environment assignments to be set for the execution. + * + * @generated from field: repeated flyteidl.core.ExecutionEnvAssignment execution_env_assignments = 9; + */ + executionEnvAssignments: ExecutionEnvAssignment[] = []; + constructor(data?: PartialMessage) { super(); proto3.util.initPartial(data, this); @@ -515,6 +523,7 @@ export class WorkflowExecutionConfig extends Message { { no: 6, name: "interruptible", kind: "message", T: BoolValue }, { no: 7, name: "overwrite_cache", kind: "scalar", T: 8 /* ScalarType.BOOL */ }, { no: 8, name: "envs", kind: "message", T: Envs }, + { no: 9, name: "execution_env_assignments", kind: "message", T: ExecutionEnvAssignment, repeated: true }, ]); static fromBinary(bytes: Uint8Array, options?: Partial): WorkflowExecutionConfig { diff --git a/flyteidl/gen/pb-es/flyteidl/core/execution_envs_pb.ts b/flyteidl/gen/pb-es/flyteidl/core/execution_envs_pb.ts new file mode 100644 index 0000000000..350528d3f9 --- /dev/null +++ b/flyteidl/gen/pb-es/flyteidl/core/execution_envs_pb.ts @@ -0,0 +1,141 @@ +// @generated by protoc-gen-es v1.7.2 with parameter "target=ts" +// @generated from file flyteidl/core/execution_envs.proto (package flyteidl.core, syntax proto3) +/* eslint-disable */ +// @ts-nocheck + +import type { BinaryReadOptions, FieldList, JsonReadOptions, JsonValue, PartialMessage, PlainMessage } from "@bufbuild/protobuf"; +import { Message, proto3, Struct } from "@bufbuild/protobuf"; + +/** + * ExecutionEnvAssignment is a message that is used to assign an execution environment to a set of + * nodes. + * + * @generated from message flyteidl.core.ExecutionEnvAssignment + */ +export class ExecutionEnvAssignment extends Message { + /** + * node_ids is a list of node ids that are being assigned the execution environment. + * + * @generated from field: repeated string node_ids = 1; + */ + nodeIds: string[] = []; + + /** + * task_type is the type of task that is being assigned. This is used to override which Flyte + * plugin will be used during execution. + * + * @generated from field: string task_type = 2; + */ + taskType = ""; + + /** + * execution_env is the environment that is being assigned to the nodes. + * + * @generated from field: flyteidl.core.ExecutionEnv execution_env = 3; + */ + executionEnv?: ExecutionEnv; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "flyteidl.core.ExecutionEnvAssignment"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "node_ids", kind: "scalar", T: 9 /* ScalarType.STRING */, repeated: true }, + { no: 2, name: "task_type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "execution_env", kind: "message", T: ExecutionEnv }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ExecutionEnvAssignment { + return new ExecutionEnvAssignment().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ExecutionEnvAssignment { + return new ExecutionEnvAssignment().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ExecutionEnvAssignment { + return new ExecutionEnvAssignment().fromJsonString(jsonString, options); + } + + static equals(a: ExecutionEnvAssignment | PlainMessage | undefined, b: ExecutionEnvAssignment | PlainMessage | undefined): boolean { + return proto3.util.equals(ExecutionEnvAssignment, a, b); + } +} + +/** + * ExecutionEnv is a message that is used to specify the execution environment. + * + * @generated from message flyteidl.core.ExecutionEnv + */ +export class ExecutionEnv extends Message { + /** + * id is a unique identifier for the execution environment. + * + * @generated from field: string id = 1; + */ + id = ""; + + /** + * type is the type of the execution environment. + * + * @generated from field: string type = 2; + */ + type = ""; + + /** + * environment is a oneof field that can be used to specify the environment in different ways. + * + * @generated from oneof flyteidl.core.ExecutionEnv.environment + */ + environment: { + /** + * extant is a reference to an existing environment. + * + * @generated from field: google.protobuf.Struct extant = 3; + */ + value: Struct; + case: "extant"; + } | { + /** + * spec is a specification of the environment. + * + * @generated from field: google.protobuf.Struct spec = 4; + */ + value: Struct; + case: "spec"; + } | { case: undefined; value?: undefined } = { case: undefined }; + + constructor(data?: PartialMessage) { + super(); + proto3.util.initPartial(data, this); + } + + static readonly runtime: typeof proto3 = proto3; + static readonly typeName = "flyteidl.core.ExecutionEnv"; + static readonly fields: FieldList = proto3.util.newFieldList(() => [ + { no: 1, name: "id", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 2, name: "type", kind: "scalar", T: 9 /* ScalarType.STRING */ }, + { no: 3, name: "extant", kind: "message", T: Struct, oneof: "environment" }, + { no: 4, name: "spec", kind: "message", T: Struct, oneof: "environment" }, + ]); + + static fromBinary(bytes: Uint8Array, options?: Partial): ExecutionEnv { + return new ExecutionEnv().fromBinary(bytes, options); + } + + static fromJson(jsonValue: JsonValue, options?: Partial): ExecutionEnv { + return new ExecutionEnv().fromJson(jsonValue, options); + } + + static fromJsonString(jsonString: string, options?: Partial): ExecutionEnv { + return new ExecutionEnv().fromJsonString(jsonString, options); + } + + static equals(a: ExecutionEnv | PlainMessage | undefined, b: ExecutionEnv | PlainMessage | undefined): boolean { + return proto3.util.equals(ExecutionEnv, a, b); + } +} + diff --git a/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go b/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go index aa67ea3820..c20946373c 100644 --- a/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/admin/execution.pb.go @@ -1267,6 +1267,8 @@ type ExecutionSpec struct { Envs *Envs `protobuf:"bytes,23,opt,name=envs,proto3" json:"envs,omitempty"` // Tags to be set for the execution. Tags []string `protobuf:"bytes,24,rep,name=tags,proto3" json:"tags,omitempty"` + // Execution environment assignments to be set for the execution. + ExecutionEnvAssignments []*core.ExecutionEnvAssignment `protobuf:"bytes,25,rep,name=execution_env_assignments,json=executionEnvAssignments,proto3" json:"execution_env_assignments,omitempty"` } func (x *ExecutionSpec) Reset() { @@ -1429,6 +1431,13 @@ func (x *ExecutionSpec) GetTags() []string { return nil } +func (x *ExecutionSpec) GetExecutionEnvAssignments() []*core.ExecutionEnvAssignment { + if x != nil { + return x.ExecutionEnvAssignments + } + return nil +} + type isExecutionSpec_NotificationOverrides interface { isExecutionSpec_NotificationOverrides() } @@ -2274,382 +2283,391 @@ var file_flyteidl_admin_execution_proto_rawDesc = []byte{ 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x6c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, - 0x72, 0x65, 0x2f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, - 0x6f, 0x72, 0x65, 0x2f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, - 0x6f, 0x72, 0x65, 0x2f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x1a, 0x1c, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, - 0x2f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 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, - 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, - 0x22, 0xd6, 0x01, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, - 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, - 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x12, 0x0a, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x12, 0x31, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, - 0x73, 0x70, 0x65, 0x63, 0x12, 0x31, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, - 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x52, - 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, 0x22, 0x99, 0x01, 0x0a, 0x18, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, - 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, - 0x69, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x43, 0x61, 0x63, 0x68, 0x65, 0x4a, - 0x04, 0x08, 0x02, 0x10, 0x03, 0x22, 0xa8, 0x01, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x12, 0x3d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, - 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x22, 0x55, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3a, 0x0a, 0x02, 0x69, + 0x6f, 0x74, 0x6f, 0x1a, 0x22, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, + 0x72, 0x65, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x76, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, + 0x69, 0x64, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, + 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 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, 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, 0x22, 0xd6, 0x01, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x6f, 0x6d, + 0x61, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, + 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x31, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, + 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x31, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, + 0x4d, 0x61, 0x70, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x6f, + 0x72, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, 0x22, 0x99, 0x01, + 0x0a, 0x18, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x6c, 0x61, 0x75, + 0x6e, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6f, 0x76, + 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x43, 0x61, + 0x63, 0x68, 0x65, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x22, 0xa8, 0x01, 0x0a, 0x17, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, + 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x22, 0x55, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x22, 0x59, 0x0a, 0x1b, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x22, 0xb6, 0x01, 0x0a, 0x09, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, + 0x12, 0x31, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, + 0x70, 0x65, 0x63, 0x12, 0x3a, 0x0a, 0x07, 0x63, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x52, 0x07, 0x63, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x22, + 0x60, 0x0a, 0x0d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, + 0x12, 0x39, 0x0a, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, + 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, + 0x6e, 0x22, 0x65, 0x0a, 0x0e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, + 0x6c, 0x6f, 0x62, 0x12, 0x37, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, + 0x18, 0x01, 0x48, 0x00, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x12, 0x0a, 0x03, + 0x75, 0x72, 0x69, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x03, 0x75, 0x72, 0x69, + 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x43, 0x0a, 0x0d, 0x41, 0x62, 0x6f, 0x72, + 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x61, 0x75, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x12, + 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x22, 0x98, 0x07, + 0x0a, 0x10, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x75, + 0x72, 0x65, 0x12, 0x3e, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, + 0x6c, 0x6f, 0x62, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x73, 0x12, 0x35, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x48, 0x00, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x25, 0x0a, 0x0b, 0x61, 0x62, 0x6f, + 0x72, 0x74, 0x5f, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, + 0x18, 0x01, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x62, 0x6f, 0x72, 0x74, 0x43, 0x61, 0x75, 0x73, 0x65, + 0x12, 0x46, 0x0a, 0x0e, 0x61, 0x62, 0x6f, 0x72, 0x74, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x62, 0x6f, 0x72, 0x74, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x0d, 0x61, 0x62, 0x6f, 0x72, 0x74, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, + 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x0a, + 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, 0x46, 0x0a, 0x0f, 0x63, 0x6f, + 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, + 0x18, 0x01, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x73, 0x12, 0x3c, 0x0a, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x26, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x68, 0x61, 0x73, 0x65, 0x52, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, + 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x35, 0x0a, 0x08, 0x64, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x08, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, + 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, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, + 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x42, 0x0a, 0x0d, 0x6e, 0x6f, 0x74, 0x69, + 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, + 0x2e, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6e, + 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3a, 0x0a, 0x0b, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0a, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x12, 0x5d, 0x0a, 0x14, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x52, 0x12, 0x73, 0x74, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, + 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x0f, 0x0a, 0x0d, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x5b, 0x0a, 0x0e, 0x53, 0x79, 0x73, 0x74, + 0x65, 0x6d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1c, 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, 0x22, 0x85, 0x05, 0x0a, 0x11, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x43, 0x0a, 0x04, 0x6d, + 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x6d, 0x6f, 0x64, 0x65, + 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x12, 0x18, + 0x0a, 0x07, 0x6e, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x07, 0x6e, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0c, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x64, 0x41, 0x74, 0x12, 0x5a, 0x0a, 0x15, 0x70, 0x61, 0x72, 0x65, 0x6e, + 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x13, + 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, 0x13, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x12, 0x72, 0x65, + 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x47, 0x0a, 0x0f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x53, 0x79, 0x73, 0x74, 0x65, + 0x6d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0e, 0x73, 0x79, 0x73, 0x74, 0x65, + 0x6d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x3c, 0x0a, 0x0c, 0x61, 0x72, 0x74, + 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x12, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, + 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x44, 0x52, 0x0b, 0x61, 0x72, 0x74, 0x69, + 0x66, 0x61, 0x63, 0x74, 0x49, 0x64, 0x73, 0x22, 0x74, 0x0a, 0x0d, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4d, 0x41, 0x4e, 0x55, + 0x41, 0x4c, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, + 0x44, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x59, 0x53, 0x54, 0x45, 0x4d, 0x10, 0x02, 0x12, + 0x0c, 0x0a, 0x08, 0x52, 0x45, 0x4c, 0x41, 0x55, 0x4e, 0x43, 0x48, 0x10, 0x03, 0x12, 0x12, 0x0a, + 0x0e, 0x43, 0x48, 0x49, 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x10, + 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, 0x45, 0x44, 0x10, 0x05, + 0x12, 0x0b, 0x0a, 0x07, 0x54, 0x52, 0x49, 0x47, 0x47, 0x45, 0x52, 0x10, 0x06, 0x22, 0x56, 0x0a, + 0x10, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, + 0x74, 0x12, 0x42, 0x0a, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xf3, 0x08, 0x0a, 0x0d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x3a, 0x0a, 0x0b, 0x6c, 0x61, 0x75, 0x6e, 0x63, + 0x68, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0a, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, + 0x6c, 0x61, 0x6e, 0x12, 0x35, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, + 0x18, 0x01, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x3d, 0x0a, 0x08, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, + 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x48, 0x0a, 0x0d, 0x6e, 0x6f, 0x74, + 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x20, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, + 0x6e, 0x2e, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, + 0x73, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0b, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x61, + 0x6c, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0a, 0x64, 0x69, 0x73, 0x61, + 0x62, 0x6c, 0x65, 0x41, 0x6c, 0x6c, 0x12, 0x2e, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x52, 0x06, + 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x3d, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x6e, 0x6e, + 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x49, 0x0a, 0x10, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, + 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, + 0x53, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, + 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, + 0x12, 0x39, 0x0a, 0x09, 0x61, 0x75, 0x74, 0x68, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x10, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x52, 0x6f, 0x6c, 0x65, 0x42, 0x02, 0x18, + 0x01, 0x52, 0x08, 0x61, 0x75, 0x74, 0x68, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x4d, 0x0a, 0x12, 0x71, + 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, + 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x10, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, + 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6d, 0x61, + 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x12, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, + 0x69, 0x73, 0x6d, 0x12, 0x58, 0x0a, 0x16, 0x72, 0x61, 0x77, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x13, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x52, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, + 0x74, 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x13, 0x72, 0x61, 0x77, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x50, 0x0a, + 0x12, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x11, 0x63, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, + 0x40, 0x0a, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 0x65, + 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x42, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x52, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, + 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x63, + 0x61, 0x63, 0x68, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6f, 0x76, 0x65, 0x72, + 0x77, 0x72, 0x69, 0x74, 0x65, 0x43, 0x61, 0x63, 0x68, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x65, 0x6e, + 0x76, 0x73, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x6e, 0x76, 0x73, 0x52, 0x04, + 0x65, 0x6e, 0x76, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x18, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x61, 0x0a, 0x19, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x76, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x19, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x52, 0x17, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x18, 0x0a, 0x16, 0x6e, + 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x76, 0x65, 0x72, + 0x72, 0x69, 0x64, 0x65, 0x73, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x22, 0x6d, 0x0a, 0x19, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x22, 0x1c, 0x0a, 0x1a, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5d, 0x0a, 0x1f, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, + 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x22, 0x59, 0x0a, 0x1b, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, - 0x69, 0x64, 0x22, 0xb6, 0x01, 0x0a, 0x09, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x31, 0x0a, 0x04, - 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, - 0x3a, 0x0a, 0x07, 0x63, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x20, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, - 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x75, - 0x72, 0x65, 0x52, 0x07, 0x63, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x22, 0x60, 0x0a, 0x0d, 0x45, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x39, 0x0a, 0x0a, - 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, - 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x65, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x65, 0x0a, - 0x0e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x6c, 0x6f, 0x62, 0x12, - 0x37, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, - 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x12, 0x0a, 0x03, 0x75, 0x72, 0x69, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x03, 0x75, 0x72, 0x69, 0x42, 0x06, 0x0a, 0x04, - 0x64, 0x61, 0x74, 0x61, 0x22, 0x43, 0x0a, 0x0d, 0x41, 0x62, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x70, - 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x22, 0x98, 0x07, 0x0a, 0x10, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x12, 0x3e, - 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x6c, 0x6f, 0x62, 0x42, - 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x12, 0x35, - 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x05, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x25, 0x0a, 0x0b, 0x61, 0x62, 0x6f, 0x72, 0x74, 0x5f, 0x63, - 0x61, 0x75, 0x73, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, - 0x52, 0x0a, 0x61, 0x62, 0x6f, 0x72, 0x74, 0x43, 0x61, 0x75, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x0e, - 0x61, 0x62, 0x6f, 0x72, 0x74, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0c, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, - 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x62, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x0d, 0x61, 0x62, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x64, - 0x61, 0x74, 0x61, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, - 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, - 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x0a, 0x6f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, 0x46, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, - 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, 0x18, 0x01, 0x52, 0x0e, - 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x64, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x3c, - 0x0a, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x50, 0x68, 0x61, 0x73, 0x65, 0x52, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x12, 0x39, 0x0a, 0x0a, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x35, 0x0a, 0x08, 0x64, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 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, 0x08, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x39, - 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, - 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x64, 0x41, 0x74, 0x12, 0x42, 0x0a, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4e, 0x6f, 0x74, - 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3a, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, - 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x49, 0x64, 0x12, 0x5d, 0x0a, 0x14, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x68, - 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x0e, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, - 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, - 0x12, 0x73, 0x74, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x65, 0x74, 0x61, - 0x69, 0x6c, 0x73, 0x42, 0x0f, 0x0a, 0x0d, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x22, 0x5b, 0x0a, 0x0e, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x10, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x12, 0x1c, 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, 0x22, 0x85, 0x05, 0x0a, 0x11, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x43, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x09, - 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x6e, 0x65, - 0x73, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x6e, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, - 0x64, 0x5f, 0x61, 0x74, 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, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, - 0x64, 0x41, 0x74, 0x12, 0x5a, 0x0a, 0x15, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x6f, - 0x64, 0x65, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x13, 0x70, 0x61, 0x72, 0x65, - 0x6e, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x5b, 0x0a, 0x13, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x65, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x12, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, - 0x6e, 0x63, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x0f, - 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, - 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x3c, 0x0a, 0x0c, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, - 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x12, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x41, 0x72, 0x74, 0x69, - 0x66, 0x61, 0x63, 0x74, 0x49, 0x44, 0x52, 0x0b, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, - 0x49, 0x64, 0x73, 0x22, 0x74, 0x0a, 0x0d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4d, 0x41, 0x4e, 0x55, 0x41, 0x4c, 0x10, 0x00, - 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x43, 0x48, 0x45, 0x44, 0x55, 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, - 0x0a, 0x0a, 0x06, 0x53, 0x59, 0x53, 0x54, 0x45, 0x4d, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x52, - 0x45, 0x4c, 0x41, 0x55, 0x4e, 0x43, 0x48, 0x10, 0x03, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x48, 0x49, - 0x4c, 0x44, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x04, 0x12, 0x0d, 0x0a, - 0x09, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, 0x45, 0x44, 0x10, 0x05, 0x12, 0x0b, 0x0a, 0x07, - 0x54, 0x52, 0x49, 0x47, 0x47, 0x45, 0x52, 0x10, 0x06, 0x22, 0x56, 0x0a, 0x10, 0x4e, 0x6f, 0x74, - 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x42, 0x0a, - 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, - 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x22, 0x90, 0x08, 0x0a, 0x0d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, - 0x70, 0x65, 0x63, 0x12, 0x3a, 0x0a, 0x0b, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x5f, 0x70, 0x6c, - 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x52, 0x0a, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x12, - 0x35, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x42, 0x02, 0x18, 0x01, 0x52, 0x06, - 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x3d, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x22, 0x88, 0x02, 0x0a, 0x20, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, + 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, 0x0a, 0x07, + 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x55, + 0x72, 0x6c, 0x42, 0x6c, 0x6f, 0x62, 0x42, 0x02, 0x18, 0x01, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x73, 0x12, 0x33, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x55, 0x72, 0x6c, 0x42, 0x6c, 0x6f, 0x62, 0x42, 0x02, 0x18, 0x01, + 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x3a, 0x0a, 0x0b, 0x66, 0x75, 0x6c, 0x6c, + 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, + 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x52, 0x0a, 0x66, 0x75, 0x6c, 0x6c, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x73, 0x12, 0x3c, 0x0a, 0x0c, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x6f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, + 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, + 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x52, 0x0b, 0x66, 0x75, 0x6c, 0x6c, 0x4f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x73, 0x22, 0x8a, 0x01, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x34, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x48, 0x0a, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4e, 0x6f, - 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x48, 0x00, - 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, - 0x21, 0x0a, 0x0b, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0a, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x41, - 0x6c, 0x6c, 0x12, 0x2e, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, - 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, - 0x6c, 0x73, 0x12, 0x3d, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, - 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x12, 0x49, 0x0a, 0x10, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x63, 0x6f, - 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x53, 0x65, 0x63, 0x75, - 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0f, 0x73, 0x65, 0x63, - 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x39, 0x0a, 0x09, - 0x61, 0x75, 0x74, 0x68, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x18, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x2e, 0x41, 0x75, 0x74, 0x68, 0x52, 0x6f, 0x6c, 0x65, 0x42, 0x02, 0x18, 0x01, 0x52, 0x08, 0x61, - 0x75, 0x74, 0x68, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x4d, 0x0a, 0x12, 0x71, 0x75, 0x61, 0x6c, 0x69, - 0x74, 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x11, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, - 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x52, 0x10, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, - 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x12, 0x20, 0x01, 0x28, 0x05, 0x52, - 0x0e, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x12, - 0x58, 0x0a, 0x16, 0x72, 0x61, 0x77, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x64, 0x61, - 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x23, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x2e, 0x52, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x52, 0x13, 0x72, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, - 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x50, 0x0a, 0x12, 0x63, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, - 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x41, 0x73, - 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x11, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, - 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x40, 0x0a, 0x0d, 0x69, - 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x18, 0x15, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x42, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0d, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x12, 0x27, 0x0a, - 0x0f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, - 0x18, 0x16, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, - 0x65, 0x43, 0x61, 0x63, 0x68, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x17, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, - 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x6e, 0x76, 0x73, 0x52, 0x04, 0x65, 0x6e, 0x76, 0x73, - 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x18, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, - 0x74, 0x61, 0x67, 0x73, 0x42, 0x18, 0x0a, 0x16, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x4a, 0x04, - 0x08, 0x04, 0x10, 0x05, 0x22, 0x6d, 0x0a, 0x19, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, + 0xae, 0x01, 0x0a, 0x1b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x12, + 0x34, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, + 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x3b, 0x0a, 0x0b, 0x6f, 0x63, 0x63, 0x75, 0x72, 0x72, 0x65, + 0x64, 0x5f, 0x61, 0x74, 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, 0x6f, 0x63, 0x63, 0x75, 0x72, 0x72, 0x65, 0x64, + 0x41, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, + 0x22, 0x19, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x76, 0x0a, 0x22, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, - 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x61, - 0x75, 0x73, 0x65, 0x22, 0x1c, 0x0a, 0x1a, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x5d, 0x0a, 0x1f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, - 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, - 0x22, 0x88, 0x02, 0x0a, 0x20, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, - 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x55, 0x72, 0x6c, 0x42, 0x6c, 0x6f, 0x62, 0x42, - 0x02, 0x18, 0x01, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x12, 0x33, 0x0a, 0x06, - 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x55, 0x72, - 0x6c, 0x42, 0x6c, 0x6f, 0x62, 0x42, 0x02, 0x18, 0x01, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, - 0x73, 0x12, 0x3a, 0x0a, 0x0b, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, - 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, - 0x70, 0x52, 0x0a, 0x66, 0x75, 0x6c, 0x6c, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x3c, 0x0a, - 0x0c, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, - 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x52, 0x0b, - 0x66, 0x75, 0x6c, 0x6c, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, 0x8a, 0x01, 0x0a, 0x16, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, - 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, - 0x69, 0x64, 0x12, 0x34, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, - 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xae, 0x01, 0x0a, 0x1b, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, - 0x65, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x12, 0x34, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, - 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x3b, - 0x0a, 0x0b, 0x6f, 0x63, 0x63, 0x75, 0x72, 0x72, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x6f, 0x63, 0x63, 0x75, 0x72, 0x72, 0x65, 0x64, 0x41, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x70, - 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x70, 0x72, 0x69, 0x6e, 0x63, 0x69, 0x70, 0x61, 0x6c, 0x22, 0x19, 0x0a, 0x17, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x76, 0x0a, 0x22, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, - 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x02, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, - 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x22, 0x4e, 0x0a, 0x23, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x27, 0x0a, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x53, 0x70, 0x61, 0x6e, 0x52, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x22, 0x79, 0x0a, 0x19, + 0x05, 0x64, 0x65, 0x70, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x64, 0x65, + 0x70, 0x74, 0x68, 0x22, 0x4e, 0x0a, 0x23, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x0a, 0x04, 0x73, 0x70, + 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x53, 0x70, 0x61, 0x6e, 0x52, 0x04, 0x73, + 0x70, 0x61, 0x6e, 0x22, 0x79, 0x0a, 0x19, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x18, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x6f, + 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, + 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6f, 0x72, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0x6c, + 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, + 0x73, 0x42, 0x79, 0x50, 0x68, 0x61, 0x73, 0x65, 0x12, 0x3c, 0x0a, 0x05, 0x70, 0x68, 0x61, 0x73, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x68, 0x61, 0x73, 0x65, 0x52, + 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x6f, 0x0a, 0x1a, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x47, - 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x72, 0x6f, - 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x6a, - 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6f, - 0x72, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, 0x12, 0x18, 0x0a, - 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0x6c, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x50, 0x68, 0x61, 0x73, - 0x65, 0x12, 0x3c, 0x0a, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x26, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, - 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x2e, 0x50, 0x68, 0x61, 0x73, 0x65, 0x52, 0x05, 0x70, 0x68, 0x61, 0x73, 0x65, 0x12, - 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x6f, 0x0a, 0x1a, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x51, 0x0a, 0x10, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x42, 0x79, - 0x50, 0x68, 0x61, 0x73, 0x65, 0x52, 0x0f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x22, 0x66, 0x0a, 0x20, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, - 0x67, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, - 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x72, - 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, 0x6f, - 0x6a, 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, - 0x6f, 0x72, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, 0x22, 0x39, - 0x0a, 0x21, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x2a, 0x3e, 0x0a, 0x0e, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x45, - 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, - 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, - 0x52, 0x43, 0x48, 0x49, 0x56, 0x45, 0x44, 0x10, 0x01, 0x42, 0xba, 0x01, 0x0a, 0x12, 0x63, 0x6f, - 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x42, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, - 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, - 0x02, 0x03, 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, - 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, - 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, - 0x3a, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x51, 0x0a, 0x10, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x50, 0x68, 0x61, 0x73, 0x65, 0x52, 0x0f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x22, 0x66, 0x0a, + 0x20, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x64, + 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, + 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6f, 0x72, 0x67, 0x22, 0x39, 0x0a, 0x21, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, 0x67, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x47, + 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, + 0x2a, 0x3e, 0x0a, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x52, 0x43, 0x48, 0x49, 0x56, 0x45, 0x44, 0x10, 0x01, + 0x42, 0xba, 0x01, 0x0a, 0x12, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x42, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, + 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, + 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, + 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, 0x02, 0x03, 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, + 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, + 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, + 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2716,8 +2734,9 @@ var file_flyteidl_admin_execution_proto_goTypes = []interface{}{ (*ClusterAssignment)(nil), // 46: flyteidl.admin.ClusterAssignment (*wrapperspb.BoolValue)(nil), // 47: google.protobuf.BoolValue (*Envs)(nil), // 48: flyteidl.admin.Envs - (*UrlBlob)(nil), // 49: flyteidl.admin.UrlBlob - (*core.Span)(nil), // 50: flyteidl.core.Span + (*core.ExecutionEnvAssignment)(nil), // 49: flyteidl.core.ExecutionEnvAssignment + (*UrlBlob)(nil), // 50: flyteidl.admin.UrlBlob + (*core.Span)(nil), // 51: flyteidl.core.Span } var file_flyteidl_admin_execution_proto_depIdxs = []int32{ 15, // 0: flyteidl.admin.ExecutionCreateRequest.spec:type_name -> flyteidl.admin.ExecutionSpec @@ -2765,25 +2784,26 @@ var file_flyteidl_admin_execution_proto_depIdxs = []int32{ 46, // 42: flyteidl.admin.ExecutionSpec.cluster_assignment:type_name -> flyteidl.admin.ClusterAssignment 47, // 43: flyteidl.admin.ExecutionSpec.interruptible:type_name -> google.protobuf.BoolValue 48, // 44: flyteidl.admin.ExecutionSpec.envs:type_name -> flyteidl.admin.Envs - 31, // 45: flyteidl.admin.ExecutionTerminateRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier - 31, // 46: flyteidl.admin.WorkflowExecutionGetDataRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier - 49, // 47: flyteidl.admin.WorkflowExecutionGetDataResponse.outputs:type_name -> flyteidl.admin.UrlBlob - 49, // 48: flyteidl.admin.WorkflowExecutionGetDataResponse.inputs:type_name -> flyteidl.admin.UrlBlob - 30, // 49: flyteidl.admin.WorkflowExecutionGetDataResponse.full_inputs:type_name -> flyteidl.core.LiteralMap - 30, // 50: flyteidl.admin.WorkflowExecutionGetDataResponse.full_outputs:type_name -> flyteidl.core.LiteralMap - 31, // 51: flyteidl.admin.ExecutionUpdateRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier - 0, // 52: flyteidl.admin.ExecutionUpdateRequest.state:type_name -> flyteidl.admin.ExecutionState - 0, // 53: flyteidl.admin.ExecutionStateChangeDetails.state:type_name -> flyteidl.admin.ExecutionState - 34, // 54: flyteidl.admin.ExecutionStateChangeDetails.occurred_at:type_name -> google.protobuf.Timestamp - 31, // 55: flyteidl.admin.WorkflowExecutionGetMetricsRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier - 50, // 56: flyteidl.admin.WorkflowExecutionGetMetricsResponse.span:type_name -> flyteidl.core.Span - 33, // 57: flyteidl.admin.ExecutionCountsByPhase.phase:type_name -> flyteidl.core.WorkflowExecution.Phase - 26, // 58: flyteidl.admin.ExecutionCountsGetResponse.execution_counts:type_name -> flyteidl.admin.ExecutionCountsByPhase - 59, // [59:59] is the sub-list for method output_type - 59, // [59:59] is the sub-list for method input_type - 59, // [59:59] is the sub-list for extension type_name - 59, // [59:59] is the sub-list for extension extendee - 0, // [0:59] is the sub-list for field type_name + 49, // 45: flyteidl.admin.ExecutionSpec.execution_env_assignments:type_name -> flyteidl.core.ExecutionEnvAssignment + 31, // 46: flyteidl.admin.ExecutionTerminateRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier + 31, // 47: flyteidl.admin.WorkflowExecutionGetDataRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier + 50, // 48: flyteidl.admin.WorkflowExecutionGetDataResponse.outputs:type_name -> flyteidl.admin.UrlBlob + 50, // 49: flyteidl.admin.WorkflowExecutionGetDataResponse.inputs:type_name -> flyteidl.admin.UrlBlob + 30, // 50: flyteidl.admin.WorkflowExecutionGetDataResponse.full_inputs:type_name -> flyteidl.core.LiteralMap + 30, // 51: flyteidl.admin.WorkflowExecutionGetDataResponse.full_outputs:type_name -> flyteidl.core.LiteralMap + 31, // 52: flyteidl.admin.ExecutionUpdateRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier + 0, // 53: flyteidl.admin.ExecutionUpdateRequest.state:type_name -> flyteidl.admin.ExecutionState + 0, // 54: flyteidl.admin.ExecutionStateChangeDetails.state:type_name -> flyteidl.admin.ExecutionState + 34, // 55: flyteidl.admin.ExecutionStateChangeDetails.occurred_at:type_name -> google.protobuf.Timestamp + 31, // 56: flyteidl.admin.WorkflowExecutionGetMetricsRequest.id:type_name -> flyteidl.core.WorkflowExecutionIdentifier + 51, // 57: flyteidl.admin.WorkflowExecutionGetMetricsResponse.span:type_name -> flyteidl.core.Span + 33, // 58: flyteidl.admin.ExecutionCountsByPhase.phase:type_name -> flyteidl.core.WorkflowExecution.Phase + 26, // 59: flyteidl.admin.ExecutionCountsGetResponse.execution_counts:type_name -> flyteidl.admin.ExecutionCountsByPhase + 60, // [60:60] is the sub-list for method output_type + 60, // [60:60] is the sub-list for method input_type + 60, // [60:60] is the sub-list for extension type_name + 60, // [60:60] is the sub-list for extension extendee + 0, // [0:60] is the sub-list for field type_name } func init() { file_flyteidl_admin_execution_proto_init() } diff --git a/flyteidl/gen/pb-go/flyteidl/admin/launch_plan.pb.go b/flyteidl/gen/pb-go/flyteidl/admin/launch_plan.pb.go index 1bbd7a29e1..eb72b91044 100644 --- a/flyteidl/gen/pb-go/flyteidl/admin/launch_plan.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/admin/launch_plan.pb.go @@ -415,6 +415,8 @@ type LaunchPlanSpec struct { OverwriteCache bool `protobuf:"varint,20,opt,name=overwrite_cache,json=overwriteCache,proto3" json:"overwrite_cache,omitempty"` // Environment variables to be set for the execution. Envs *Envs `protobuf:"bytes,21,opt,name=envs,proto3" json:"envs,omitempty"` + // Execution environment assignments to be set for the execution. + ExecutionEnvAssignments []*core.ExecutionEnvAssignment `protobuf:"bytes,22,rep,name=execution_env_assignments,json=executionEnvAssignments,proto3" json:"execution_env_assignments,omitempty"` } func (x *LaunchPlanSpec) Reset() { @@ -564,6 +566,13 @@ func (x *LaunchPlanSpec) GetEnvs() *Envs { return nil } +func (x *LaunchPlanSpec) GetExecutionEnvAssignments() []*core.ExecutionEnvAssignment { + if x != nil { + return x.ExecutionEnvAssignments + } + return nil +} + // Values computed by the flyte platform after launch plan registration. // These include expected_inputs required to be present in a CreateExecutionRequest // to launch the reference workflow as well timestamp values associated with the launch plan. @@ -978,194 +987,202 @@ var file_flyteidl_admin_launch_plan_proto_rawDesc = []byte{ 0x74, 0x6f, 0x12, 0x0e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x1a, 0x1d, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x1a, 0x1c, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, - 0x2f, 0x6c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, - 0x1e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x69, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, - 0x1d, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x69, - 0x6e, 0x74, 0x65, 0x72, 0x66, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x73, 0x65, - 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2f, 0x73, 0x63, 0x68, - 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 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, 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, 0x22, 0x78, 0x0a, 0x17, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, - 0x61, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x29, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x32, 0x0a, 0x04, 0x73, 0x70, - 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, - 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x22, 0x1a, - 0x0a, 0x18, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa8, 0x01, 0x0a, 0x0a, 0x4c, - 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x29, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x52, 0x02, 0x69, 0x64, 0x12, 0x32, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, - 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x70, - 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x3b, 0x0a, 0x07, 0x63, 0x6c, 0x6f, 0x73, - 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, - 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, - 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x52, 0x07, 0x63, 0x6c, - 0x6f, 0x73, 0x75, 0x72, 0x65, 0x22, 0x65, 0x0a, 0x0e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, - 0x6c, 0x61, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x3d, 0x0a, 0x0c, 0x6c, 0x61, 0x75, 0x6e, 0x63, - 0x68, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, - 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x0b, 0x6c, 0x61, 0x75, 0x6e, 0x63, - 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x76, 0x0a, 0x04, - 0x41, 0x75, 0x74, 0x68, 0x12, 0x2c, 0x0a, 0x12, 0x61, 0x73, 0x73, 0x75, 0x6d, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x69, 0x61, 0x6d, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x10, 0x61, 0x73, 0x73, 0x75, 0x6d, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x61, 0x6d, 0x52, 0x6f, - 0x6c, 0x65, 0x12, 0x3c, 0x0a, 0x1a, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, - 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, - 0x65, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, - 0x3a, 0x02, 0x18, 0x01, 0x22, 0xbd, 0x07, 0x0a, 0x0e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, - 0x6c, 0x61, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x3a, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x49, 0x64, 0x12, 0x4b, 0x0a, 0x0f, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x5f, 0x6d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, - 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x52, 0x0e, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x12, 0x42, 0x0a, 0x0e, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x6e, 0x70, 0x75, - 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, - 0x65, 0x72, 0x4d, 0x61, 0x70, 0x52, 0x0d, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x73, 0x12, 0x3c, 0x0a, 0x0c, 0x66, 0x69, 0x78, 0x65, 0x64, 0x5f, 0x69, 0x6e, - 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, - 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x52, 0x0b, 0x66, 0x69, 0x78, 0x65, 0x64, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x73, 0x12, 0x16, 0x0a, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x42, 0x02, 0x18, 0x01, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x12, 0x2e, 0x0a, 0x06, 0x6c, 0x61, - 0x62, 0x65, 0x6c, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x62, 0x65, - 0x6c, 0x73, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x3d, 0x0a, 0x0b, 0x61, 0x6e, - 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x61, 0x6e, - 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x04, 0x61, 0x75, 0x74, - 0x68, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, - 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x42, 0x02, 0x18, - 0x01, 0x52, 0x04, 0x61, 0x75, 0x74, 0x68, 0x12, 0x39, 0x0a, 0x09, 0x61, 0x75, 0x74, 0x68, 0x5f, - 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x75, 0x74, 0x68, - 0x52, 0x6f, 0x6c, 0x65, 0x42, 0x02, 0x18, 0x01, 0x52, 0x08, 0x61, 0x75, 0x74, 0x68, 0x52, 0x6f, - 0x6c, 0x65, 0x12, 0x49, 0x0a, 0x10, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, 0x63, - 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x53, 0x65, 0x63, - 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0f, 0x73, 0x65, - 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x4d, 0x0a, - 0x12, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, - 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, - 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x10, 0x71, 0x75, 0x61, 0x6c, - 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x58, 0x0a, 0x16, - 0x72, 0x61, 0x77, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x52, 0x61, - 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x52, 0x13, 0x72, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x27, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, - 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x12, 0x20, 0x01, 0x28, 0x05, 0x52, - 0x0e, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x12, - 0x40, 0x0a, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 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, 0x42, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, - 0x75, 0x65, 0x52, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, - 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x63, - 0x61, 0x63, 0x68, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6f, 0x76, 0x65, 0x72, - 0x77, 0x72, 0x69, 0x74, 0x65, 0x43, 0x61, 0x63, 0x68, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x65, 0x6e, - 0x76, 0x73, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x6e, 0x76, 0x73, 0x52, 0x04, - 0x65, 0x6e, 0x76, 0x73, 0x22, 0xcd, 0x02, 0x0a, 0x11, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, - 0x6c, 0x61, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, - 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, - 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, - 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x44, 0x0a, 0x0f, 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, - 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, - 0x65, 0x74, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x52, 0x0e, 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, - 0x64, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x45, 0x0a, 0x10, 0x65, 0x78, 0x70, 0x65, 0x63, - 0x74, 0x65, 0x64, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, - 0x65, 0x2e, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x61, 0x70, 0x52, 0x0f, 0x65, - 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, 0x64, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x12, 0x39, - 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x75, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x64, 0x41, 0x74, 0x22, 0xd1, 0x01, 0x0a, 0x12, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, - 0x6c, 0x61, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x34, 0x0a, 0x08, 0x73, - 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x53, - 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x08, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, - 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x41, 0x0a, 0x11, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x5f, - 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 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, 0x10, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x43, 0x6f, - 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x7b, 0x0a, 0x17, 0x4c, 0x61, 0x75, 0x6e, - 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x6f, 0x1a, 0x22, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, + 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x76, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, + 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x6c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, + 0x72, 0x65, 0x2f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, + 0x72, 0x65, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x66, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x1c, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, + 0x65, 0x2f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x1d, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, + 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x1b, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2f, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 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, 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, 0x22, 0x78, 0x0a, 0x17, 0x4c, 0x61, 0x75, 0x6e, + 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, - 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x35, - 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x32, + 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, + 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, + 0x65, 0x63, 0x22, 0x1a, 0x0a, 0x18, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa8, + 0x01, 0x0a, 0x0a, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x29, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x32, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, + 0x61, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x3b, 0x0a, 0x07, + 0x63, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, - 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x1a, 0x0a, 0x18, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, - 0x6c, 0x61, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x50, 0x0a, 0x17, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x4c, 0x61, 0x75, 0x6e, 0x63, - 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x35, 0x0a, 0x02, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x45, - 0x6e, 0x74, 0x69, 0x74, 0x79, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, - 0x02, 0x69, 0x64, 0x22, 0xbc, 0x01, 0x0a, 0x1b, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x4c, 0x61, - 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, - 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, - 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x74, - 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, - 0x6e, 0x12, 0x2d, 0x0a, 0x07, 0x73, 0x6f, 0x72, 0x74, 0x5f, 0x62, 0x79, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, - 0x6d, 0x69, 0x6e, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x52, 0x06, 0x73, 0x6f, 0x72, 0x74, 0x42, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, - 0x72, 0x67, 0x2a, 0x2b, 0x0a, 0x0f, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x4e, 0x41, 0x43, 0x54, 0x49, 0x56, - 0x45, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x01, 0x42, - 0xbb, 0x01, 0x0a, 0x12, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x42, 0x0f, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, - 0x61, 0x6e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, - 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, - 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, 0x02, 0x03, 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, - 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, - 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, - 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, + 0x52, 0x07, 0x63, 0x6c, 0x6f, 0x73, 0x75, 0x72, 0x65, 0x22, 0x65, 0x0a, 0x0e, 0x4c, 0x61, 0x75, + 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x3d, 0x0a, 0x0c, 0x6c, + 0x61, 0x75, 0x6e, 0x63, 0x68, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, + 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x0b, 0x6c, + 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, + 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x22, 0x76, 0x0a, 0x04, 0x41, 0x75, 0x74, 0x68, 0x12, 0x2c, 0x0a, 0x12, 0x61, 0x73, 0x73, 0x75, + 0x6d, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x61, 0x6d, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x73, 0x75, 0x6d, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x61, 0x6d, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x3c, 0x0a, 0x1a, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, + 0x65, 0x74, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x6b, 0x75, 0x62, 0x65, + 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x3a, 0x02, 0x18, 0x01, 0x22, 0xa0, 0x08, 0x0a, 0x0e, 0x4c, 0x61, 0x75, + 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x3a, 0x0a, 0x0b, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, + 0x2e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0a, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x12, 0x4b, 0x0a, 0x0f, 0x65, 0x6e, 0x74, 0x69, 0x74, + 0x79, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x22, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, + 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x52, 0x0e, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x12, 0x42, 0x0a, 0x0e, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x52, 0x0d, 0x64, 0x65, 0x66, 0x61, 0x75, + 0x6c, 0x74, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x3c, 0x0a, 0x0c, 0x66, 0x69, 0x78, 0x65, + 0x64, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, + 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x4c, + 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4d, 0x61, 0x70, 0x52, 0x0b, 0x66, 0x69, 0x78, 0x65, 0x64, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x16, 0x0a, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x12, 0x2e, + 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, + 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, + 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x3d, + 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x0a, + 0x04, 0x61, 0x75, 0x74, 0x68, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, 0x75, 0x74, + 0x68, 0x42, 0x02, 0x18, 0x01, 0x52, 0x04, 0x61, 0x75, 0x74, 0x68, 0x12, 0x39, 0x0a, 0x09, 0x61, + 0x75, 0x74, 0x68, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, + 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, + 0x41, 0x75, 0x74, 0x68, 0x52, 0x6f, 0x6c, 0x65, 0x42, 0x02, 0x18, 0x01, 0x52, 0x08, 0x61, 0x75, + 0x74, 0x68, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x49, 0x0a, 0x10, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, + 0x74, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, + 0x2e, 0x53, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, + 0x52, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, + 0x74, 0x12, 0x4d, 0x0a, 0x12, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x6f, 0x66, 0x5f, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, + 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x10, + 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x12, 0x58, 0x0a, 0x16, 0x72, 0x61, 0x77, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x64, + 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x23, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, + 0x6e, 0x2e, 0x52, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x13, 0x72, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x27, 0x0a, 0x0f, 0x6d, 0x61, + 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x12, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, + 0x69, 0x73, 0x6d, 0x12, 0x40, 0x0a, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, + 0x69, 0x62, 0x6c, 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, 0x42, 0x6f, 0x6f, + 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, + 0x74, 0x69, 0x62, 0x6c, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, + 0x74, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, + 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x43, 0x61, 0x63, 0x68, 0x65, 0x12, 0x28, + 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x6e, + 0x76, 0x73, 0x52, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x12, 0x61, 0x0a, 0x19, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x76, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x16, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x52, 0x17, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xcd, 0x02, 0x0a, 0x11, + 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x43, 0x6c, 0x6f, 0x73, 0x75, 0x72, + 0x65, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, + 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x44, 0x0a, 0x0f, 0x65, 0x78, 0x70, 0x65, + 0x63, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x52, 0x0e, + 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, 0x64, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x45, + 0x0a, 0x10, 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, + 0x65, 0x4d, 0x61, 0x70, 0x52, 0x0f, 0x65, 0x78, 0x70, 0x65, 0x63, 0x74, 0x65, 0x64, 0x4f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, + 0x5f, 0x61, 0x74, 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, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, + 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x22, 0xd1, 0x01, 0x0a, 0x12, + 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x12, 0x34, 0x0a, 0x08, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x52, 0x08, + 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x6e, 0x6f, 0x74, 0x69, + 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, + 0x2e, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6e, + 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x41, 0x0a, 0x11, + 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x5f, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x03, 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, 0x10, 0x6c, + 0x61, 0x75, 0x6e, 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, + 0x7b, 0x0a, 0x17, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x52, 0x02, 0x69, 0x64, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x1a, 0x0a, 0x18, + 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x50, 0x0a, 0x17, 0x41, 0x63, 0x74, 0x69, + 0x76, 0x65, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x35, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x25, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, + 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x45, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x02, 0x69, 0x64, 0x22, 0xbc, 0x01, 0x0a, 0x1b, 0x41, + 0x63, 0x74, 0x69, 0x76, 0x65, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x4c, + 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x72, + 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, 0x6f, + 0x6a, 0x65, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x14, 0x0a, 0x05, + 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, + 0x69, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x2d, 0x0a, 0x07, 0x73, 0x6f, 0x72, 0x74, + 0x5f, 0x62, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x52, + 0x06, 0x73, 0x6f, 0x72, 0x74, 0x42, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, 0x2a, 0x2b, 0x0a, 0x0f, 0x4c, 0x61, 0x75, + 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0c, 0x0a, 0x08, + 0x49, 0x4e, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, + 0x54, 0x49, 0x56, 0x45, 0x10, 0x01, 0x42, 0xbb, 0x01, 0x0a, 0x12, 0x63, 0x6f, 0x6d, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x42, 0x0f, 0x4c, + 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, + 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, + 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, + 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, 0x02, 0x03, + 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x41, + 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, + 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, + 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x41, + 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1207,13 +1224,14 @@ var file_flyteidl_admin_launch_plan_proto_goTypes = []interface{}{ (*RawOutputDataConfig)(nil), // 21: flyteidl.admin.RawOutputDataConfig (*wrapperspb.BoolValue)(nil), // 22: google.protobuf.BoolValue (*Envs)(nil), // 23: flyteidl.admin.Envs - (*core.VariableMap)(nil), // 24: flyteidl.core.VariableMap - (*timestamppb.Timestamp)(nil), // 25: google.protobuf.Timestamp - (*Schedule)(nil), // 26: flyteidl.admin.Schedule - (*Notification)(nil), // 27: flyteidl.admin.Notification - (*anypb.Any)(nil), // 28: google.protobuf.Any - (*NamedEntityIdentifier)(nil), // 29: flyteidl.admin.NamedEntityIdentifier - (*Sort)(nil), // 30: flyteidl.admin.Sort + (*core.ExecutionEnvAssignment)(nil), // 24: flyteidl.core.ExecutionEnvAssignment + (*core.VariableMap)(nil), // 25: flyteidl.core.VariableMap + (*timestamppb.Timestamp)(nil), // 26: google.protobuf.Timestamp + (*Schedule)(nil), // 27: flyteidl.admin.Schedule + (*Notification)(nil), // 28: flyteidl.admin.Notification + (*anypb.Any)(nil), // 29: google.protobuf.Any + (*NamedEntityIdentifier)(nil), // 30: flyteidl.admin.NamedEntityIdentifier + (*Sort)(nil), // 31: flyteidl.admin.Sort } var file_flyteidl_admin_launch_plan_proto_depIdxs = []int32{ 13, // 0: flyteidl.admin.LaunchPlanCreateRequest.id:type_name -> flyteidl.core.Identifier @@ -1235,23 +1253,24 @@ var file_flyteidl_admin_launch_plan_proto_depIdxs = []int32{ 21, // 16: flyteidl.admin.LaunchPlanSpec.raw_output_data_config:type_name -> flyteidl.admin.RawOutputDataConfig 22, // 17: flyteidl.admin.LaunchPlanSpec.interruptible:type_name -> google.protobuf.BoolValue 23, // 18: flyteidl.admin.LaunchPlanSpec.envs:type_name -> flyteidl.admin.Envs - 0, // 19: flyteidl.admin.LaunchPlanClosure.state:type_name -> flyteidl.admin.LaunchPlanState - 14, // 20: flyteidl.admin.LaunchPlanClosure.expected_inputs:type_name -> flyteidl.core.ParameterMap - 24, // 21: flyteidl.admin.LaunchPlanClosure.expected_outputs:type_name -> flyteidl.core.VariableMap - 25, // 22: flyteidl.admin.LaunchPlanClosure.created_at:type_name -> google.protobuf.Timestamp - 25, // 23: flyteidl.admin.LaunchPlanClosure.updated_at:type_name -> google.protobuf.Timestamp - 26, // 24: flyteidl.admin.LaunchPlanMetadata.schedule:type_name -> flyteidl.admin.Schedule - 27, // 25: flyteidl.admin.LaunchPlanMetadata.notifications:type_name -> flyteidl.admin.Notification - 28, // 26: flyteidl.admin.LaunchPlanMetadata.launch_conditions:type_name -> google.protobuf.Any - 13, // 27: flyteidl.admin.LaunchPlanUpdateRequest.id:type_name -> flyteidl.core.Identifier - 0, // 28: flyteidl.admin.LaunchPlanUpdateRequest.state:type_name -> flyteidl.admin.LaunchPlanState - 29, // 29: flyteidl.admin.ActiveLaunchPlanRequest.id:type_name -> flyteidl.admin.NamedEntityIdentifier - 30, // 30: flyteidl.admin.ActiveLaunchPlanListRequest.sort_by:type_name -> flyteidl.admin.Sort - 31, // [31:31] is the sub-list for method output_type - 31, // [31:31] is the sub-list for method input_type - 31, // [31:31] is the sub-list for extension type_name - 31, // [31:31] is the sub-list for extension extendee - 0, // [0:31] is the sub-list for field type_name + 24, // 19: flyteidl.admin.LaunchPlanSpec.execution_env_assignments:type_name -> flyteidl.core.ExecutionEnvAssignment + 0, // 20: flyteidl.admin.LaunchPlanClosure.state:type_name -> flyteidl.admin.LaunchPlanState + 14, // 21: flyteidl.admin.LaunchPlanClosure.expected_inputs:type_name -> flyteidl.core.ParameterMap + 25, // 22: flyteidl.admin.LaunchPlanClosure.expected_outputs:type_name -> flyteidl.core.VariableMap + 26, // 23: flyteidl.admin.LaunchPlanClosure.created_at:type_name -> google.protobuf.Timestamp + 26, // 24: flyteidl.admin.LaunchPlanClosure.updated_at:type_name -> google.protobuf.Timestamp + 27, // 25: flyteidl.admin.LaunchPlanMetadata.schedule:type_name -> flyteidl.admin.Schedule + 28, // 26: flyteidl.admin.LaunchPlanMetadata.notifications:type_name -> flyteidl.admin.Notification + 29, // 27: flyteidl.admin.LaunchPlanMetadata.launch_conditions:type_name -> google.protobuf.Any + 13, // 28: flyteidl.admin.LaunchPlanUpdateRequest.id:type_name -> flyteidl.core.Identifier + 0, // 29: flyteidl.admin.LaunchPlanUpdateRequest.state:type_name -> flyteidl.admin.LaunchPlanState + 30, // 30: flyteidl.admin.ActiveLaunchPlanRequest.id:type_name -> flyteidl.admin.NamedEntityIdentifier + 31, // 31: flyteidl.admin.ActiveLaunchPlanListRequest.sort_by:type_name -> flyteidl.admin.Sort + 32, // [32:32] is the sub-list for method output_type + 32, // [32:32] is the sub-list for method input_type + 32, // [32:32] is the sub-list for extension type_name + 32, // [32:32] is the sub-list for extension extendee + 0, // [0:32] is the sub-list for field type_name } func init() { file_flyteidl_admin_launch_plan_proto_init() } diff --git a/flyteidl/gen/pb-go/flyteidl/admin/matchable_resource.pb.go b/flyteidl/gen/pb-go/flyteidl/admin/matchable_resource.pb.go index 2b712042d9..9e51abfb31 100644 --- a/flyteidl/gen/pb-go/flyteidl/admin/matchable_resource.pb.go +++ b/flyteidl/gen/pb-go/flyteidl/admin/matchable_resource.pb.go @@ -569,6 +569,8 @@ type WorkflowExecutionConfig struct { OverwriteCache bool `protobuf:"varint,7,opt,name=overwrite_cache,json=overwriteCache,proto3" json:"overwrite_cache,omitempty"` // Environment variables to be set for the execution. Envs *Envs `protobuf:"bytes,8,opt,name=envs,proto3" json:"envs,omitempty"` + // Execution environment assignments to be set for the execution. + ExecutionEnvAssignments []*core.ExecutionEnvAssignment `protobuf:"bytes,9,rep,name=execution_env_assignments,json=executionEnvAssignments,proto3" json:"execution_env_assignments,omitempty"` } func (x *WorkflowExecutionConfig) Reset() { @@ -659,6 +661,13 @@ func (x *WorkflowExecutionConfig) GetEnvs() *Envs { return nil } +func (x *WorkflowExecutionConfig) GetExecutionEnvAssignments() []*core.ExecutionEnvAssignment { + if x != nil { + return x.ExecutionEnvAssignments + } + return nil +} + // Generic container for encapsulating all types of the above attributes messages. type MatchingAttributes struct { state protoimpl.MessageState @@ -1037,205 +1046,213 @@ var file_flyteidl_admin_matchable_resource_proto_rawDesc = []byte{ 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x65, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x73, 0x65, - 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 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, 0x22, 0x95, 0x01, 0x0a, - 0x10, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x70, 0x65, - 0x63, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x70, 0x75, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x63, 0x70, 0x75, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x70, 0x75, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x67, 0x70, 0x75, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x18, 0x0a, - 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x70, 0x68, 0x65, 0x6d, - 0x65, 0x72, 0x61, 0x6c, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x10, 0x65, 0x70, 0x68, 0x65, 0x6d, 0x65, 0x72, 0x61, 0x6c, 0x53, 0x74, 0x6f, - 0x72, 0x61, 0x67, 0x65, 0x22, 0x90, 0x01, 0x0a, 0x16, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, - 0x3c, 0x0a, 0x08, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x22, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x76, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x1c, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, + 0x65, 0x2f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 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, + 0x22, 0x95, 0x01, 0x0a, 0x10, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x70, 0x75, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x63, 0x70, 0x75, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x70, 0x75, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x67, 0x70, 0x75, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x6d, + 0x6f, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, + 0x79, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x65, + 0x70, 0x68, 0x65, 0x6d, 0x65, 0x72, 0x61, 0x6c, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x65, 0x70, 0x68, 0x65, 0x6d, 0x65, 0x72, 0x61, + 0x6c, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x22, 0x90, 0x01, 0x0a, 0x16, 0x54, 0x61, 0x73, + 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x08, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, + 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x08, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x73, 0x12, 0x38, 0x0a, 0x06, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, - 0x70, 0x65, 0x63, 0x52, 0x08, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x38, 0x0a, - 0x06, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x54, - 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, - 0x06, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x22, 0xb5, 0x01, 0x0a, 0x19, 0x43, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, - 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x59, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, - 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x66, 0x6c, 0x79, 0x74, - 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, - 0x75, 0x74, 0x65, 0x73, 0x2e, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, - 0x1a, 0x3d, 0x0a, 0x0f, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0x2e, 0x0a, 0x18, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x51, 0x75, 0x65, 0x75, - 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x74, - 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x22, - 0x2d, 0x0a, 0x15, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xec, - 0x01, 0x0a, 0x0e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, - 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, - 0x0a, 0x09, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x08, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x6c, 0x0a, 0x17, 0x6d, - 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x62, 0x65, - 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x34, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x50, 0x6c, - 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x2e, 0x4d, 0x69, 0x73, - 0x73, 0x69, 0x6e, 0x67, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, - 0x6f, 0x72, 0x52, 0x15, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x50, 0x6c, 0x75, 0x67, 0x69, - 0x6e, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0x32, 0x0a, 0x15, 0x4d, 0x69, 0x73, - 0x73, 0x69, 0x6e, 0x67, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, - 0x6f, 0x72, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x41, 0x49, 0x4c, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, - 0x55, 0x53, 0x45, 0x5f, 0x44, 0x45, 0x46, 0x41, 0x55, 0x4c, 0x54, 0x10, 0x01, 0x22, 0x4f, 0x0a, - 0x0f, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, - 0x12, 0x3c, 0x0a, 0x09, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, - 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, - 0x69, 0x64, 0x65, 0x52, 0x09, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x22, 0xeb, - 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, 0x27, 0x0a, 0x0f, 0x6d, 0x61, - 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, - 0x69, 0x73, 0x6d, 0x12, 0x49, 0x0a, 0x10, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x5f, - 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x53, 0x65, - 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0f, 0x73, - 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x58, - 0x0a, 0x16, 0x72, 0x61, 0x77, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x64, 0x61, 0x74, - 0x61, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, - 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, - 0x52, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x52, 0x13, 0x72, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, - 0x74, 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2e, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, - 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, - 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x3d, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x41, - 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x40, 0x0a, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 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, 0x42, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0d, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x6f, 0x76, 0x65, - 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x43, 0x61, 0x63, - 0x68, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, - 0x6e, 0x2e, 0x45, 0x6e, 0x76, 0x73, 0x52, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x22, 0x94, 0x06, 0x0a, - 0x12, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, - 0x74, 0x65, 0x73, 0x12, 0x62, 0x0a, 0x18, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x72, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, - 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, - 0x16, 0x74, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, - 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x6b, 0x0a, 0x1b, 0x63, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, 0x6c, - 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, - 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x19, 0x63, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, - 0x75, 0x74, 0x65, 0x73, 0x12, 0x68, 0x0a, 0x1a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, - 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, - 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x69, 0x6f, 0x6e, 0x51, 0x75, 0x65, 0x75, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, - 0x65, 0x73, 0x48, 0x00, 0x52, 0x18, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x51, - 0x75, 0x65, 0x75, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x5f, - 0x0a, 0x17, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x25, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, - 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x48, 0x00, 0x52, 0x15, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, - 0x4f, 0x0a, 0x12, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x65, - 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, - 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, - 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x48, 0x00, 0x52, 0x10, - 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x12, 0x4c, 0x0a, 0x10, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, - 0x69, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, - 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x50, 0x6c, 0x75, 0x67, - 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x70, - 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x12, 0x65, - 0x0a, 0x19, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x27, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, - 0x69, 0x6e, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x17, 0x77, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x52, 0x0a, 0x12, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, - 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, - 0x69, 0x6e, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x11, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, - 0x67, 0x65, 0x74, 0x22, 0xe7, 0x01, 0x0a, 0x20, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, - 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x42, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x66, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4d, 0x61, - 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, - 0x52, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, - 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, - 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x1a, - 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x61, - 0x75, 0x6e, 0x63, 0x68, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6f, - 0x72, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, 0x22, 0x7a, 0x0a, - 0x1e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x74, - 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x46, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, - 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, - 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, 0x22, 0x7b, 0x0a, 0x1f, 0x4c, 0x69, 0x73, - 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, - 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x58, 0x0a, 0x0e, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, - 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x41, - 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2a, 0xe0, 0x01, 0x0a, 0x11, 0x4d, 0x61, 0x74, 0x63, 0x68, - 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x11, 0x0a, 0x0d, - 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x10, 0x00, 0x12, - 0x14, 0x0a, 0x10, 0x43, 0x4c, 0x55, 0x53, 0x54, 0x45, 0x52, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, - 0x52, 0x43, 0x45, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, - 0x4f, 0x4e, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x10, 0x02, 0x12, 0x1b, 0x0a, 0x17, 0x45, 0x58, - 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4c, 0x55, 0x53, 0x54, 0x45, 0x52, 0x5f, - 0x4c, 0x41, 0x42, 0x45, 0x4c, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20, 0x51, 0x55, 0x41, 0x4c, 0x49, - 0x54, 0x59, 0x5f, 0x4f, 0x46, 0x5f, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x5f, 0x53, 0x50, - 0x45, 0x43, 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x04, 0x12, 0x13, 0x0a, - 0x0f, 0x50, 0x4c, 0x55, 0x47, 0x49, 0x4e, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x52, 0x49, 0x44, 0x45, - 0x10, 0x05, 0x12, 0x1d, 0x0a, 0x19, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, - 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x10, - 0x06, 0x12, 0x16, 0x0a, 0x12, 0x43, 0x4c, 0x55, 0x53, 0x54, 0x45, 0x52, 0x5f, 0x41, 0x53, 0x53, - 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x07, 0x42, 0xc2, 0x01, 0x0a, 0x12, 0x63, 0x6f, - 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, - 0x42, 0x16, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, - 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, - 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, - 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, - 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, 0x02, 0x03, 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, - 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, - 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, - 0x02, 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, - 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, - 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x70, 0x65, 0x63, 0x52, 0x06, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x22, 0xb5, 0x01, 0x0a, 0x19, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x59, 0x0a, 0x0a, 0x61, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x2e, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x2e, 0x0a, 0x18, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, + 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, + 0x61, 0x67, 0x73, 0x22, 0x2d, 0x0a, 0x15, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x22, 0xec, 0x01, 0x0a, 0x0e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, + 0x72, 0x72, 0x69, 0x64, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x49, 0x64, 0x12, + 0x6c, 0x0a, 0x17, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x6c, 0x75, 0x67, 0x69, + 0x6e, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x34, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, + 0x6e, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, + 0x2e, 0x4d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x42, 0x65, + 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x15, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x50, + 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0x32, 0x0a, + 0x15, 0x4d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x42, 0x65, + 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x41, 0x49, 0x4c, 0x10, 0x00, + 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x53, 0x45, 0x5f, 0x44, 0x45, 0x46, 0x41, 0x55, 0x4c, 0x54, 0x10, + 0x01, 0x22, 0x4f, 0x0a, 0x0f, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, + 0x69, 0x64, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x09, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, + 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, + 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x52, 0x09, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, + 0x65, 0x73, 0x22, 0xce, 0x04, 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, 0x27, + 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, + 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, + 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x12, 0x49, 0x0a, 0x10, 0x73, 0x65, 0x63, 0x75, 0x72, + 0x69, 0x74, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, + 0x65, 0x2e, 0x53, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, + 0x74, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, + 0x78, 0x74, 0x12, 0x58, 0x0a, 0x16, 0x72, 0x61, 0x77, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, + 0x6d, 0x69, 0x6e, 0x2e, 0x52, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x44, 0x61, 0x74, + 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x13, 0x72, 0x61, 0x77, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2e, 0x0a, 0x06, + 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4c, 0x61, + 0x62, 0x65, 0x6c, 0x73, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x3d, 0x0a, 0x0b, + 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, + 0x69, 0x6e, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, + 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x40, 0x0a, 0x0d, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 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, 0x42, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0d, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x12, 0x27, 0x0a, + 0x0f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, + 0x65, 0x43, 0x61, 0x63, 0x68, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x6e, 0x76, 0x73, 0x52, 0x04, 0x65, 0x6e, 0x76, 0x73, + 0x12, 0x61, 0x0a, 0x19, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, + 0x76, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x09, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x17, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x22, 0x94, 0x06, 0x0a, 0x12, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x62, 0x0a, 0x18, 0x74, 0x61, + 0x73, 0x6b, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x66, + 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x54, 0x61, + 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x16, 0x74, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x6b, + 0x0a, 0x1b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, + 0x52, 0x19, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x68, 0x0a, 0x1a, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x61, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x28, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, + 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x51, 0x75, 0x65, 0x75, 0x65, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x48, 0x00, 0x52, 0x18, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x51, 0x75, 0x65, 0x75, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x5f, 0x0a, 0x17, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x48, 0x00, 0x52, + 0x15, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x4f, 0x0a, 0x12, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, + 0x79, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x2e, 0x51, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x48, 0x00, 0x52, 0x10, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x66, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x4c, 0x0a, 0x10, 0x70, 0x6c, 0x75, 0x67, 0x69, + 0x6e, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1f, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, + 0x69, 0x6e, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, + 0x65, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x4f, 0x76, 0x65, 0x72, + 0x72, 0x69, 0x64, 0x65, 0x73, 0x12, 0x65, 0x0a, 0x19, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x48, 0x00, 0x52, 0x17, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x52, 0x0a, 0x12, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, + 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x11, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xe7, 0x01, 0x0a, 0x20, 0x4d, + 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x42, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, + 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x69, 0x6e, 0x67, 0x41, 0x74, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, + 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x72, + 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x5f, 0x70, 0x6c, 0x61, 0x6e, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x50, 0x6c, + 0x61, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6f, 0x72, 0x67, 0x22, 0x7a, 0x0a, 0x1e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x74, 0x63, + 0x68, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4d, + 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, + 0x0a, 0x03, 0x6f, 0x72, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6f, 0x72, 0x67, + 0x22, 0x7b, 0x0a, 0x1f, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, + 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x58, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2a, 0xe0, 0x01, + 0x0a, 0x11, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x52, 0x45, 0x53, 0x4f, + 0x55, 0x52, 0x43, 0x45, 0x10, 0x00, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x4c, 0x55, 0x53, 0x54, 0x45, + 0x52, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x51, 0x55, 0x45, 0x55, 0x45, 0x10, + 0x02, 0x12, 0x1b, 0x0a, 0x17, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, + 0x4c, 0x55, 0x53, 0x54, 0x45, 0x52, 0x5f, 0x4c, 0x41, 0x42, 0x45, 0x4c, 0x10, 0x03, 0x12, 0x24, + 0x0a, 0x20, 0x51, 0x55, 0x41, 0x4c, 0x49, 0x54, 0x59, 0x5f, 0x4f, 0x46, 0x5f, 0x53, 0x45, 0x52, + 0x56, 0x49, 0x43, 0x45, 0x5f, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, + 0x4f, 0x4e, 0x10, 0x04, 0x12, 0x13, 0x0a, 0x0f, 0x50, 0x4c, 0x55, 0x47, 0x49, 0x4e, 0x5f, 0x4f, + 0x56, 0x45, 0x52, 0x52, 0x49, 0x44, 0x45, 0x10, 0x05, 0x12, 0x1d, 0x0a, 0x19, 0x57, 0x4f, 0x52, + 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x10, 0x06, 0x12, 0x16, 0x0a, 0x12, 0x43, 0x4c, 0x55, 0x53, + 0x54, 0x45, 0x52, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x07, + 0x42, 0xc2, 0x01, 0x0a, 0x12, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x2e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x42, 0x16, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x61, 0x62, + 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, + 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, + 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, + 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0xa2, 0x02, + 0x03, 0x46, 0x41, 0x58, 0xaa, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, + 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xca, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, + 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0xe2, 0x02, 0x1a, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x5c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0xea, 0x02, 0x0f, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x3a, 0x3a, + 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1274,8 +1291,9 @@ var file_flyteidl_admin_matchable_resource_proto_goTypes = []interface{}{ (*Annotations)(nil), // 18: flyteidl.admin.Annotations (*wrapperspb.BoolValue)(nil), // 19: google.protobuf.BoolValue (*Envs)(nil), // 20: flyteidl.admin.Envs - (*core.QualityOfService)(nil), // 21: flyteidl.core.QualityOfService - (*ClusterAssignment)(nil), // 22: flyteidl.admin.ClusterAssignment + (*core.ExecutionEnvAssignment)(nil), // 21: flyteidl.core.ExecutionEnvAssignment + (*core.QualityOfService)(nil), // 22: flyteidl.core.QualityOfService + (*ClusterAssignment)(nil), // 23: flyteidl.admin.ClusterAssignment } var file_flyteidl_admin_matchable_resource_proto_depIdxs = []int32{ 2, // 0: flyteidl.admin.TaskResourceAttributes.defaults:type_name -> flyteidl.admin.TaskResourceSpec @@ -1289,22 +1307,23 @@ var file_flyteidl_admin_matchable_resource_proto_depIdxs = []int32{ 18, // 8: flyteidl.admin.WorkflowExecutionConfig.annotations:type_name -> flyteidl.admin.Annotations 19, // 9: flyteidl.admin.WorkflowExecutionConfig.interruptible:type_name -> google.protobuf.BoolValue 20, // 10: flyteidl.admin.WorkflowExecutionConfig.envs:type_name -> flyteidl.admin.Envs - 3, // 11: flyteidl.admin.MatchingAttributes.task_resource_attributes:type_name -> flyteidl.admin.TaskResourceAttributes - 4, // 12: flyteidl.admin.MatchingAttributes.cluster_resource_attributes:type_name -> flyteidl.admin.ClusterResourceAttributes - 5, // 13: flyteidl.admin.MatchingAttributes.execution_queue_attributes:type_name -> flyteidl.admin.ExecutionQueueAttributes - 6, // 14: flyteidl.admin.MatchingAttributes.execution_cluster_label:type_name -> flyteidl.admin.ExecutionClusterLabel - 21, // 15: flyteidl.admin.MatchingAttributes.quality_of_service:type_name -> flyteidl.core.QualityOfService - 8, // 16: flyteidl.admin.MatchingAttributes.plugin_overrides:type_name -> flyteidl.admin.PluginOverrides - 9, // 17: flyteidl.admin.MatchingAttributes.workflow_execution_config:type_name -> flyteidl.admin.WorkflowExecutionConfig - 22, // 18: flyteidl.admin.MatchingAttributes.cluster_assignment:type_name -> flyteidl.admin.ClusterAssignment - 10, // 19: flyteidl.admin.MatchableAttributesConfiguration.attributes:type_name -> flyteidl.admin.MatchingAttributes - 0, // 20: flyteidl.admin.ListMatchableAttributesRequest.resource_type:type_name -> flyteidl.admin.MatchableResource - 11, // 21: flyteidl.admin.ListMatchableAttributesResponse.configurations:type_name -> flyteidl.admin.MatchableAttributesConfiguration - 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 + 21, // 11: flyteidl.admin.WorkflowExecutionConfig.execution_env_assignments:type_name -> flyteidl.core.ExecutionEnvAssignment + 3, // 12: flyteidl.admin.MatchingAttributes.task_resource_attributes:type_name -> flyteidl.admin.TaskResourceAttributes + 4, // 13: flyteidl.admin.MatchingAttributes.cluster_resource_attributes:type_name -> flyteidl.admin.ClusterResourceAttributes + 5, // 14: flyteidl.admin.MatchingAttributes.execution_queue_attributes:type_name -> flyteidl.admin.ExecutionQueueAttributes + 6, // 15: flyteidl.admin.MatchingAttributes.execution_cluster_label:type_name -> flyteidl.admin.ExecutionClusterLabel + 22, // 16: flyteidl.admin.MatchingAttributes.quality_of_service:type_name -> flyteidl.core.QualityOfService + 8, // 17: flyteidl.admin.MatchingAttributes.plugin_overrides:type_name -> flyteidl.admin.PluginOverrides + 9, // 18: flyteidl.admin.MatchingAttributes.workflow_execution_config:type_name -> flyteidl.admin.WorkflowExecutionConfig + 23, // 19: flyteidl.admin.MatchingAttributes.cluster_assignment:type_name -> flyteidl.admin.ClusterAssignment + 10, // 20: flyteidl.admin.MatchableAttributesConfiguration.attributes:type_name -> flyteidl.admin.MatchingAttributes + 0, // 21: flyteidl.admin.ListMatchableAttributesRequest.resource_type:type_name -> flyteidl.admin.MatchableResource + 11, // 22: flyteidl.admin.ListMatchableAttributesResponse.configurations:type_name -> flyteidl.admin.MatchableAttributesConfiguration + 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_flyteidl_admin_matchable_resource_proto_init() } diff --git a/flyteidl/gen/pb-go/flyteidl/core/execution_envs.pb.go b/flyteidl/gen/pb-go/flyteidl/core/execution_envs.pb.go new file mode 100644 index 0000000000..1384843c87 --- /dev/null +++ b/flyteidl/gen/pb-go/flyteidl/core/execution_envs.pb.go @@ -0,0 +1,321 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.30.0 +// protoc (unknown) +// source: flyteidl/core/execution_envs.proto + +package core + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + structpb "google.golang.org/protobuf/types/known/structpb" + reflect "reflect" + sync "sync" +) + +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) +) + +// ExecutionEnvAssignment is a message that is used to assign an execution environment to a set of +// nodes. +type ExecutionEnvAssignment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // node_ids is a list of node ids that are being assigned the execution environment. + NodeIds []string `protobuf:"bytes,1,rep,name=node_ids,json=nodeIds,proto3" json:"node_ids,omitempty"` + // task_type is the type of task that is being assigned. This is used to override which Flyte + // plugin will be used during execution. + TaskType string `protobuf:"bytes,2,opt,name=task_type,json=taskType,proto3" json:"task_type,omitempty"` + // execution_env is the environment that is being assigned to the nodes. + ExecutionEnv *ExecutionEnv `protobuf:"bytes,3,opt,name=execution_env,json=executionEnv,proto3" json:"execution_env,omitempty"` +} + +func (x *ExecutionEnvAssignment) Reset() { + *x = ExecutionEnvAssignment{} + if protoimpl.UnsafeEnabled { + mi := &file_flyteidl_core_execution_envs_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExecutionEnvAssignment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutionEnvAssignment) ProtoMessage() {} + +func (x *ExecutionEnvAssignment) ProtoReflect() protoreflect.Message { + mi := &file_flyteidl_core_execution_envs_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 ExecutionEnvAssignment.ProtoReflect.Descriptor instead. +func (*ExecutionEnvAssignment) Descriptor() ([]byte, []int) { + return file_flyteidl_core_execution_envs_proto_rawDescGZIP(), []int{0} +} + +func (x *ExecutionEnvAssignment) GetNodeIds() []string { + if x != nil { + return x.NodeIds + } + return nil +} + +func (x *ExecutionEnvAssignment) GetTaskType() string { + if x != nil { + return x.TaskType + } + return "" +} + +func (x *ExecutionEnvAssignment) GetExecutionEnv() *ExecutionEnv { + if x != nil { + return x.ExecutionEnv + } + return nil +} + +// ExecutionEnv is a message that is used to specify the execution environment. +type ExecutionEnv struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // id is a unique identifier for the execution environment. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // type is the type of the execution environment. + Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + // environment is a oneof field that can be used to specify the environment in different ways. + // + // Types that are assignable to Environment: + // + // *ExecutionEnv_Extant + // *ExecutionEnv_Spec + Environment isExecutionEnv_Environment `protobuf_oneof:"environment"` +} + +func (x *ExecutionEnv) Reset() { + *x = ExecutionEnv{} + if protoimpl.UnsafeEnabled { + mi := &file_flyteidl_core_execution_envs_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExecutionEnv) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutionEnv) ProtoMessage() {} + +func (x *ExecutionEnv) ProtoReflect() protoreflect.Message { + mi := &file_flyteidl_core_execution_envs_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 ExecutionEnv.ProtoReflect.Descriptor instead. +func (*ExecutionEnv) Descriptor() ([]byte, []int) { + return file_flyteidl_core_execution_envs_proto_rawDescGZIP(), []int{1} +} + +func (x *ExecutionEnv) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *ExecutionEnv) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (m *ExecutionEnv) GetEnvironment() isExecutionEnv_Environment { + if m != nil { + return m.Environment + } + return nil +} + +func (x *ExecutionEnv) GetExtant() *structpb.Struct { + if x, ok := x.GetEnvironment().(*ExecutionEnv_Extant); ok { + return x.Extant + } + return nil +} + +func (x *ExecutionEnv) GetSpec() *structpb.Struct { + if x, ok := x.GetEnvironment().(*ExecutionEnv_Spec); ok { + return x.Spec + } + return nil +} + +type isExecutionEnv_Environment interface { + isExecutionEnv_Environment() +} + +type ExecutionEnv_Extant struct { + // extant is a reference to an existing environment. + Extant *structpb.Struct `protobuf:"bytes,3,opt,name=extant,proto3,oneof"` +} + +type ExecutionEnv_Spec struct { + // spec is a specification of the environment. + Spec *structpb.Struct `protobuf:"bytes,4,opt,name=spec,proto3,oneof"` +} + +func (*ExecutionEnv_Extant) isExecutionEnv_Environment() {} + +func (*ExecutionEnv_Spec) isExecutionEnv_Environment() {} + +var File_flyteidl_core_execution_envs_proto protoreflect.FileDescriptor + +var file_flyteidl_core_execution_envs_proto_rawDesc = []byte{ + 0x0a, 0x22, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x76, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, + 0x6f, 0x72, 0x65, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0x92, 0x01, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, + 0x6e, 0x76, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x19, 0x0a, 0x08, + 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, + 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x64, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61, 0x73, 0x6b, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x61, 0x73, 0x6b, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x40, 0x0a, 0x0d, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x65, 0x6e, 0x76, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, 0x52, 0x0c, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, 0x22, 0xa3, 0x01, 0x0a, 0x0c, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x31, 0x0a, 0x06, 0x65, + 0x78, 0x74, 0x61, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, + 0x72, 0x75, 0x63, 0x74, 0x48, 0x00, 0x52, 0x06, 0x65, 0x78, 0x74, 0x61, 0x6e, 0x74, 0x12, 0x2d, + 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, + 0x74, 0x72, 0x75, 0x63, 0x74, 0x48, 0x00, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x42, 0x0d, 0x0a, + 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0xb8, 0x01, 0x0a, + 0x11, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x63, 0x6f, + 0x72, 0x65, 0x42, 0x12, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x76, + 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, + 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x6f, 0x72, 0x67, 0x2f, 0x66, 0x6c, + 0x79, 0x74, 0x65, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, 0x67, 0x65, 0x6e, + 0x2f, 0x70, 0x62, 0x2d, 0x67, 0x6f, 0x2f, 0x66, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2f, + 0x63, 0x6f, 0x72, 0x65, 0xa2, 0x02, 0x03, 0x46, 0x43, 0x58, 0xaa, 0x02, 0x0d, 0x46, 0x6c, 0x79, + 0x74, 0x65, 0x69, 0x64, 0x6c, 0x2e, 0x43, 0x6f, 0x72, 0x65, 0xca, 0x02, 0x0d, 0x46, 0x6c, 0x79, + 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x43, 0x6f, 0x72, 0x65, 0xe2, 0x02, 0x19, 0x46, 0x6c, 0x79, + 0x74, 0x65, 0x69, 0x64, 0x6c, 0x5c, 0x43, 0x6f, 0x72, 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0e, 0x46, 0x6c, 0x79, 0x74, 0x65, 0x69, 0x64, + 0x6c, 0x3a, 0x3a, 0x43, 0x6f, 0x72, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_flyteidl_core_execution_envs_proto_rawDescOnce sync.Once + file_flyteidl_core_execution_envs_proto_rawDescData = file_flyteidl_core_execution_envs_proto_rawDesc +) + +func file_flyteidl_core_execution_envs_proto_rawDescGZIP() []byte { + file_flyteidl_core_execution_envs_proto_rawDescOnce.Do(func() { + file_flyteidl_core_execution_envs_proto_rawDescData = protoimpl.X.CompressGZIP(file_flyteidl_core_execution_envs_proto_rawDescData) + }) + return file_flyteidl_core_execution_envs_proto_rawDescData +} + +var file_flyteidl_core_execution_envs_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_flyteidl_core_execution_envs_proto_goTypes = []interface{}{ + (*ExecutionEnvAssignment)(nil), // 0: flyteidl.core.ExecutionEnvAssignment + (*ExecutionEnv)(nil), // 1: flyteidl.core.ExecutionEnv + (*structpb.Struct)(nil), // 2: google.protobuf.Struct +} +var file_flyteidl_core_execution_envs_proto_depIdxs = []int32{ + 1, // 0: flyteidl.core.ExecutionEnvAssignment.execution_env:type_name -> flyteidl.core.ExecutionEnv + 2, // 1: flyteidl.core.ExecutionEnv.extant:type_name -> google.protobuf.Struct + 2, // 2: flyteidl.core.ExecutionEnv.spec:type_name -> google.protobuf.Struct + 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_flyteidl_core_execution_envs_proto_init() } +func file_flyteidl_core_execution_envs_proto_init() { + if File_flyteidl_core_execution_envs_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_flyteidl_core_execution_envs_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ExecutionEnvAssignment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flyteidl_core_execution_envs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ExecutionEnv); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_flyteidl_core_execution_envs_proto_msgTypes[1].OneofWrappers = []interface{}{ + (*ExecutionEnv_Extant)(nil), + (*ExecutionEnv_Spec)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_flyteidl_core_execution_envs_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_flyteidl_core_execution_envs_proto_goTypes, + DependencyIndexes: file_flyteidl_core_execution_envs_proto_depIdxs, + MessageInfos: file_flyteidl_core_execution_envs_proto_msgTypes, + }.Build() + File_flyteidl_core_execution_envs_proto = out.File + file_flyteidl_core_execution_envs_proto_rawDesc = nil + file_flyteidl_core_execution_envs_proto_goTypes = nil + file_flyteidl_core_execution_envs_proto_depIdxs = nil +} diff --git a/flyteidl/gen/pb-go/gateway/flyteidl/artifacts/artifacts.swagger.json b/flyteidl/gen/pb-go/gateway/flyteidl/artifacts/artifacts.swagger.json index 9e71b64233..9b8e42f84c 100644 --- a/flyteidl/gen/pb-go/gateway/flyteidl/artifacts/artifacts.swagger.json +++ b/flyteidl/gen/pb-go/gateway/flyteidl/artifacts/artifacts.swagger.json @@ -701,6 +701,14 @@ "envs": { "$ref": "#/definitions/adminEnvs", "description": "Environment variables to be set for the execution." + }, + "execution_env_assignments": { + "type": "array", + "items": { + "type": "object", + "$ref": "#/definitions/coreExecutionEnvAssignment" + }, + "description": "Execution environment assignments to be set for the execution." } }, "description": "User-provided launch plan definition and configuration values." @@ -1169,6 +1177,49 @@ }, "description": "Represents an error thrown from a node." }, + "coreExecutionEnv": { + "type": "object", + "properties": { + "id": { + "type": "string", + "description": "id is a unique identifier for the execution environment." + }, + "type": { + "type": "string", + "description": "type is the type of the execution environment." + }, + "extant": { + "type": "object", + "description": "extant is a reference to an existing environment." + }, + "spec": { + "type": "object", + "description": "spec is a specification of the environment." + } + }, + "description": "ExecutionEnv is a message that is used to specify the execution environment." + }, + "coreExecutionEnvAssignment": { + "type": "object", + "properties": { + "node_ids": { + "type": "array", + "items": { + "type": "string" + }, + "description": "node_ids is a list of node ids that are being assigned the execution environment." + }, + "task_type": { + "type": "string", + "description": "task_type is the type of task that is being assigned. This is used to override which Flyte\nplugin will be used during execution." + }, + "execution_env": { + "$ref": "#/definitions/coreExecutionEnv", + "description": "execution_env is the environment that is being assigned to the nodes." + } + }, + "description": "ExecutionEnvAssignment is a message that is used to assign an execution environment to a set of\nnodes." + }, "coreGranularity": { "type": "string", "enum": [ diff --git a/flyteidl/gen/pb-go/gateway/flyteidl/core/execution_envs.swagger.json b/flyteidl/gen/pb-go/gateway/flyteidl/core/execution_envs.swagger.json new file mode 100644 index 0000000000..434e85e2ee --- /dev/null +++ b/flyteidl/gen/pb-go/gateway/flyteidl/core/execution_envs.swagger.json @@ -0,0 +1,46 @@ +{ + "swagger": "2.0", + "info": { + "title": "flyteidl/core/execution_envs.proto", + "version": "version not set" + }, + "consumes": [ + "application/json" + ], + "produces": [ + "application/json" + ], + "paths": {}, + "definitions": { + "googlerpcStatus": { + "type": "object", + "properties": { + "code": { + "type": "integer", + "format": "int32" + }, + "message": { + "type": "string" + }, + "details": { + "type": "array", + "items": { + "type": "object", + "$ref": "#/definitions/protobufAny" + } + } + } + }, + "protobufAny": { + "type": "object", + "properties": { + "@type": { + "type": "string", + "description": "A URL/resource name that uniquely identifies the type of the serialized\nprotocol buffer message. This string must contain at least\none \"/\" character. The last segment of the URL's path must represent\nthe fully qualified name of the type (as in\n`path/google.protobuf.Duration`). The name should be in a canonical form\n(e.g., leading \".\" is not accepted).\n\nIn practice, teams usually precompile into the binary all types that they\nexpect it to use in the context of Any. However, for URLs which use the\nscheme `http`, `https`, or no scheme, one can optionally set up a type\nserver that maps type URLs to message definitions as follows:\n\n* If no scheme is provided, `https` is assumed.\n* An HTTP GET on the URL must yield a [google.protobuf.Type][]\n value in binary format, or produce an error.\n* Applications are allowed to cache lookup results based on the\n URL, or have them precompiled into a binary to avoid any\n lookup. Therefore, binary compatibility needs to be preserved\n on changes to types. (Use versioned type names to manage\n breaking changes.)\n\nNote: this functionality is not currently available in the official\nprotobuf release, and it is not used for type URLs beginning with\ntype.googleapis.com. As of May 2023, there are no widely used type server\nimplementations and no plans to implement one.\n\nSchemes other than `http`, `https` (or the empty scheme) might be\nused with implementation specific semantics." + } + }, + "additionalProperties": {}, + "description": "`Any` contains an arbitrary serialized protocol buffer message along with a\nURL that describes the type of the serialized message.\n\nProtobuf library provides support to pack/unpack Any values in the form\nof utility functions or additional generated methods of the Any type.\n\nExample 1: Pack and unpack a message in C++.\n\n Foo foo = ...;\n Any any;\n any.PackFrom(foo);\n ...\n if (any.UnpackTo(\u0026foo)) {\n ...\n }\n\nExample 2: Pack and unpack a message in Java.\n\n Foo foo = ...;\n Any any = Any.pack(foo);\n ...\n if (any.is(Foo.class)) {\n foo = any.unpack(Foo.class);\n }\n // or ...\n if (any.isSameTypeAs(Foo.getDefaultInstance())) {\n foo = any.unpack(Foo.getDefaultInstance());\n }\n\n Example 3: Pack and unpack a message in Python.\n\n foo = Foo(...)\n any = Any()\n any.Pack(foo)\n ...\n if any.Is(Foo.DESCRIPTOR):\n any.Unpack(foo)\n ...\n\n Example 4: Pack and unpack a message in Go\n\n foo := \u0026pb.Foo{...}\n any, err := anypb.New(foo)\n if err != nil {\n ...\n }\n ...\n foo := \u0026pb.Foo{}\n if err := any.UnmarshalTo(foo); err != nil {\n ...\n }\n\nThe pack methods provided by protobuf library will by default use\n'type.googleapis.com/full.type.name' as the type URL and the unpack\nmethods only use the fully qualified type name after the last '/'\nin the type URL, for example \"foo.bar.com/x/y.z\" will yield type\nname \"y.z\".\n\nJSON\n====\nThe JSON representation of an `Any` value uses the regular\nrepresentation of the deserialized, embedded message, with an\nadditional field `@type` which contains the type URL. Example:\n\n package google.profile;\n message Person {\n string first_name = 1;\n string last_name = 2;\n }\n\n {\n \"@type\": \"type.googleapis.com/google.profile.Person\",\n \"firstName\": \u003cstring\u003e,\n \"lastName\": \u003cstring\u003e\n }\n\nIf the embedded message type is well-known and has a custom JSON\nrepresentation, that representation will be embedded adding a field\n`value` which holds the custom JSON in addition to the `@type`\nfield. Example (for message [google.protobuf.Duration][]):\n\n {\n \"@type\": \"type.googleapis.com/google.protobuf.Duration\",\n \"value\": \"1.212s\"\n }" + } + } +} diff --git a/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json b/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json index 64dcfe2554..375bf78161 100644 --- a/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json +++ b/flyteidl/gen/pb-go/gateway/flyteidl/service/admin.swagger.json @@ -9844,6 +9844,14 @@ "type": "string" }, "description": "Tags to be set for the execution." + }, + "execution_env_assignments": { + "type": "array", + "items": { + "type": "object", + "$ref": "#/definitions/coreExecutionEnvAssignment" + }, + "description": "Execution environment assignments to be set for the execution." } }, "description": "An ExecutionSpec encompasses all data used to launch this execution. The Spec does not change over the lifetime\nof an execution as it progresses across phase changes." @@ -10112,6 +10120,14 @@ "envs": { "$ref": "#/definitions/adminEnvs", "description": "Environment variables to be set for the execution." + }, + "execution_env_assignments": { + "type": "array", + "items": { + "type": "object", + "$ref": "#/definitions/coreExecutionEnvAssignment" + }, + "description": "Execution environment assignments to be set for the execution." } }, "description": "User-provided launch plan definition and configuration values." @@ -11194,6 +11210,14 @@ "envs": { "$ref": "#/definitions/adminEnvs", "description": "Environment variables to be set for the execution." + }, + "execution_env_assignments": { + "type": "array", + "items": { + "type": "object", + "$ref": "#/definitions/coreExecutionEnvAssignment" + }, + "description": "Execution environment assignments to be set for the execution." } }, "description": "Adds defaults for customizable workflow-execution specifications and overrides." @@ -11851,6 +11875,49 @@ }, "description": "Represents an error thrown from a node." }, + "coreExecutionEnv": { + "type": "object", + "properties": { + "id": { + "type": "string", + "description": "id is a unique identifier for the execution environment." + }, + "type": { + "type": "string", + "description": "type is the type of the execution environment." + }, + "extant": { + "type": "object", + "description": "extant is a reference to an existing environment." + }, + "spec": { + "type": "object", + "description": "spec is a specification of the environment." + } + }, + "description": "ExecutionEnv is a message that is used to specify the execution environment." + }, + "coreExecutionEnvAssignment": { + "type": "object", + "properties": { + "node_ids": { + "type": "array", + "items": { + "type": "string" + }, + "description": "node_ids is a list of node ids that are being assigned the execution environment." + }, + "task_type": { + "type": "string", + "description": "task_type is the type of task that is being assigned. This is used to override which Flyte\nplugin will be used during execution." + }, + "execution_env": { + "$ref": "#/definitions/coreExecutionEnv", + "description": "execution_env is the environment that is being assigned to the nodes." + } + }, + "description": "ExecutionEnvAssignment is a message that is used to assign an execution environment to a set of\nnodes." + }, "coreExecutionError": { "type": "object", "properties": { diff --git a/flyteidl/gen/pb-js/flyteidl.d.ts b/flyteidl/gen/pb-js/flyteidl.d.ts index 37648188f4..01fea0d7de 100644 --- a/flyteidl/gen/pb-js/flyteidl.d.ts +++ b/flyteidl/gen/pb-js/flyteidl.d.ts @@ -7566,6 +7566,143 @@ export namespace flyteidl { public static verify(message: { [k: string]: any }): (string|null); } + /** Properties of an ExecutionEnvAssignment. */ + interface IExecutionEnvAssignment { + + /** ExecutionEnvAssignment nodeIds */ + nodeIds?: (string[]|null); + + /** ExecutionEnvAssignment taskType */ + taskType?: (string|null); + + /** ExecutionEnvAssignment executionEnv */ + executionEnv?: (flyteidl.core.IExecutionEnv|null); + } + + /** Represents an ExecutionEnvAssignment. */ + class ExecutionEnvAssignment implements IExecutionEnvAssignment { + + /** + * Constructs a new ExecutionEnvAssignment. + * @param [properties] Properties to set + */ + constructor(properties?: flyteidl.core.IExecutionEnvAssignment); + + /** ExecutionEnvAssignment nodeIds. */ + public nodeIds: string[]; + + /** ExecutionEnvAssignment taskType. */ + public taskType: string; + + /** ExecutionEnvAssignment executionEnv. */ + public executionEnv?: (flyteidl.core.IExecutionEnv|null); + + /** + * Creates a new ExecutionEnvAssignment instance using the specified properties. + * @param [properties] Properties to set + * @returns ExecutionEnvAssignment instance + */ + public static create(properties?: flyteidl.core.IExecutionEnvAssignment): flyteidl.core.ExecutionEnvAssignment; + + /** + * Encodes the specified ExecutionEnvAssignment message. Does not implicitly {@link flyteidl.core.ExecutionEnvAssignment.verify|verify} messages. + * @param message ExecutionEnvAssignment message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: flyteidl.core.IExecutionEnvAssignment, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ExecutionEnvAssignment message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ExecutionEnvAssignment + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): flyteidl.core.ExecutionEnvAssignment; + + /** + * Verifies an ExecutionEnvAssignment message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + } + + /** Properties of an ExecutionEnv. */ + interface IExecutionEnv { + + /** ExecutionEnv id */ + id?: (string|null); + + /** ExecutionEnv type */ + type?: (string|null); + + /** ExecutionEnv extant */ + extant?: (google.protobuf.IStruct|null); + + /** ExecutionEnv spec */ + spec?: (google.protobuf.IStruct|null); + } + + /** Represents an ExecutionEnv. */ + class ExecutionEnv implements IExecutionEnv { + + /** + * Constructs a new ExecutionEnv. + * @param [properties] Properties to set + */ + constructor(properties?: flyteidl.core.IExecutionEnv); + + /** ExecutionEnv id. */ + public id: string; + + /** ExecutionEnv type. */ + public type: string; + + /** ExecutionEnv extant. */ + public extant?: (google.protobuf.IStruct|null); + + /** ExecutionEnv spec. */ + public spec?: (google.protobuf.IStruct|null); + + /** ExecutionEnv environment. */ + public environment?: ("extant"|"spec"); + + /** + * Creates a new ExecutionEnv instance using the specified properties. + * @param [properties] Properties to set + * @returns ExecutionEnv instance + */ + public static create(properties?: flyteidl.core.IExecutionEnv): flyteidl.core.ExecutionEnv; + + /** + * Encodes the specified ExecutionEnv message. Does not implicitly {@link flyteidl.core.ExecutionEnv.verify|verify} messages. + * @param message ExecutionEnv message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: flyteidl.core.IExecutionEnv, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ExecutionEnv message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ExecutionEnv + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): flyteidl.core.ExecutionEnv; + + /** + * Verifies an ExecutionEnv message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + } + /** Properties of a Span. */ interface ISpan { @@ -13922,6 +14059,9 @@ export namespace flyteidl { /** ExecutionSpec tags */ tags?: (string[]|null); + + /** ExecutionSpec executionEnvAssignments */ + executionEnvAssignments?: (flyteidl.core.IExecutionEnvAssignment[]|null); } /** Represents an ExecutionSpec. */ @@ -13984,6 +14124,9 @@ export namespace flyteidl { /** ExecutionSpec tags. */ public tags: string[]; + /** ExecutionSpec executionEnvAssignments. */ + public executionEnvAssignments: flyteidl.core.IExecutionEnvAssignment[]; + /** ExecutionSpec notificationOverrides. */ public notificationOverrides?: ("notifications"|"disableAll"); @@ -15166,6 +15309,9 @@ export namespace flyteidl { /** LaunchPlanSpec envs */ envs?: (flyteidl.admin.IEnvs|null); + + /** LaunchPlanSpec executionEnvAssignments */ + executionEnvAssignments?: (flyteidl.core.IExecutionEnvAssignment[]|null); } /** Represents a LaunchPlanSpec. */ @@ -15225,6 +15371,9 @@ export namespace flyteidl { /** LaunchPlanSpec envs. */ public envs?: (flyteidl.admin.IEnvs|null); + /** LaunchPlanSpec executionEnvAssignments. */ + public executionEnvAssignments: flyteidl.core.IExecutionEnvAssignment[]; + /** * Creates a new LaunchPlanSpec instance using the specified properties. * @param [properties] Properties to set @@ -16285,6 +16434,9 @@ export namespace flyteidl { /** WorkflowExecutionConfig envs */ envs?: (flyteidl.admin.IEnvs|null); + + /** WorkflowExecutionConfig executionEnvAssignments */ + executionEnvAssignments?: (flyteidl.core.IExecutionEnvAssignment[]|null); } /** Represents a WorkflowExecutionConfig. */ @@ -16320,6 +16472,9 @@ export namespace flyteidl { /** WorkflowExecutionConfig envs. */ public envs?: (flyteidl.admin.IEnvs|null); + /** WorkflowExecutionConfig executionEnvAssignments. */ + public executionEnvAssignments: flyteidl.core.IExecutionEnvAssignment[]; + /** * Creates a new WorkflowExecutionConfig instance using the specified properties. * @param [properties] Properties to set diff --git a/flyteidl/gen/pb-js/flyteidl.js b/flyteidl/gen/pb-js/flyteidl.js index c45ef56b40..d7528519f3 100644 --- a/flyteidl/gen/pb-js/flyteidl.js +++ b/flyteidl/gen/pb-js/flyteidl.js @@ -18286,6 +18286,348 @@ return ErrorDocument; })(); + core.ExecutionEnvAssignment = (function() { + + /** + * Properties of an ExecutionEnvAssignment. + * @memberof flyteidl.core + * @interface IExecutionEnvAssignment + * @property {Array.|null} [nodeIds] ExecutionEnvAssignment nodeIds + * @property {string|null} [taskType] ExecutionEnvAssignment taskType + * @property {flyteidl.core.IExecutionEnv|null} [executionEnv] ExecutionEnvAssignment executionEnv + */ + + /** + * Constructs a new ExecutionEnvAssignment. + * @memberof flyteidl.core + * @classdesc Represents an ExecutionEnvAssignment. + * @implements IExecutionEnvAssignment + * @constructor + * @param {flyteidl.core.IExecutionEnvAssignment=} [properties] Properties to set + */ + function ExecutionEnvAssignment(properties) { + this.nodeIds = []; + if (properties) + for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * ExecutionEnvAssignment nodeIds. + * @member {Array.} nodeIds + * @memberof flyteidl.core.ExecutionEnvAssignment + * @instance + */ + ExecutionEnvAssignment.prototype.nodeIds = $util.emptyArray; + + /** + * ExecutionEnvAssignment taskType. + * @member {string} taskType + * @memberof flyteidl.core.ExecutionEnvAssignment + * @instance + */ + ExecutionEnvAssignment.prototype.taskType = ""; + + /** + * ExecutionEnvAssignment executionEnv. + * @member {flyteidl.core.IExecutionEnv|null|undefined} executionEnv + * @memberof flyteidl.core.ExecutionEnvAssignment + * @instance + */ + ExecutionEnvAssignment.prototype.executionEnv = null; + + /** + * Creates a new ExecutionEnvAssignment instance using the specified properties. + * @function create + * @memberof flyteidl.core.ExecutionEnvAssignment + * @static + * @param {flyteidl.core.IExecutionEnvAssignment=} [properties] Properties to set + * @returns {flyteidl.core.ExecutionEnvAssignment} ExecutionEnvAssignment instance + */ + ExecutionEnvAssignment.create = function create(properties) { + return new ExecutionEnvAssignment(properties); + }; + + /** + * Encodes the specified ExecutionEnvAssignment message. Does not implicitly {@link flyteidl.core.ExecutionEnvAssignment.verify|verify} messages. + * @function encode + * @memberof flyteidl.core.ExecutionEnvAssignment + * @static + * @param {flyteidl.core.IExecutionEnvAssignment} message ExecutionEnvAssignment message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ExecutionEnvAssignment.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.nodeIds != null && message.nodeIds.length) + for (var i = 0; i < message.nodeIds.length; ++i) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.nodeIds[i]); + if (message.taskType != null && message.hasOwnProperty("taskType")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.taskType); + if (message.executionEnv != null && message.hasOwnProperty("executionEnv")) + $root.flyteidl.core.ExecutionEnv.encode(message.executionEnv, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + return writer; + }; + + /** + * Decodes an ExecutionEnvAssignment message from the specified reader or buffer. + * @function decode + * @memberof flyteidl.core.ExecutionEnvAssignment + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {flyteidl.core.ExecutionEnvAssignment} ExecutionEnvAssignment + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ExecutionEnvAssignment.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + var end = length === undefined ? reader.len : reader.pos + length, message = new $root.flyteidl.core.ExecutionEnvAssignment(); + while (reader.pos < end) { + var tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (!(message.nodeIds && message.nodeIds.length)) + message.nodeIds = []; + message.nodeIds.push(reader.string()); + break; + case 2: + message.taskType = reader.string(); + break; + case 3: + message.executionEnv = $root.flyteidl.core.ExecutionEnv.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Verifies an ExecutionEnvAssignment message. + * @function verify + * @memberof flyteidl.core.ExecutionEnvAssignment + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ExecutionEnvAssignment.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.nodeIds != null && message.hasOwnProperty("nodeIds")) { + if (!Array.isArray(message.nodeIds)) + return "nodeIds: array expected"; + for (var i = 0; i < message.nodeIds.length; ++i) + if (!$util.isString(message.nodeIds[i])) + return "nodeIds: string[] expected"; + } + if (message.taskType != null && message.hasOwnProperty("taskType")) + if (!$util.isString(message.taskType)) + return "taskType: string expected"; + if (message.executionEnv != null && message.hasOwnProperty("executionEnv")) { + var error = $root.flyteidl.core.ExecutionEnv.verify(message.executionEnv); + if (error) + return "executionEnv." + error; + } + return null; + }; + + return ExecutionEnvAssignment; + })(); + + core.ExecutionEnv = (function() { + + /** + * Properties of an ExecutionEnv. + * @memberof flyteidl.core + * @interface IExecutionEnv + * @property {string|null} [id] ExecutionEnv id + * @property {string|null} [type] ExecutionEnv type + * @property {google.protobuf.IStruct|null} [extant] ExecutionEnv extant + * @property {google.protobuf.IStruct|null} [spec] ExecutionEnv spec + */ + + /** + * Constructs a new ExecutionEnv. + * @memberof flyteidl.core + * @classdesc Represents an ExecutionEnv. + * @implements IExecutionEnv + * @constructor + * @param {flyteidl.core.IExecutionEnv=} [properties] Properties to set + */ + function ExecutionEnv(properties) { + if (properties) + for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * ExecutionEnv id. + * @member {string} id + * @memberof flyteidl.core.ExecutionEnv + * @instance + */ + ExecutionEnv.prototype.id = ""; + + /** + * ExecutionEnv type. + * @member {string} type + * @memberof flyteidl.core.ExecutionEnv + * @instance + */ + ExecutionEnv.prototype.type = ""; + + /** + * ExecutionEnv extant. + * @member {google.protobuf.IStruct|null|undefined} extant + * @memberof flyteidl.core.ExecutionEnv + * @instance + */ + ExecutionEnv.prototype.extant = null; + + /** + * ExecutionEnv spec. + * @member {google.protobuf.IStruct|null|undefined} spec + * @memberof flyteidl.core.ExecutionEnv + * @instance + */ + ExecutionEnv.prototype.spec = null; + + // OneOf field names bound to virtual getters and setters + var $oneOfFields; + + /** + * ExecutionEnv environment. + * @member {"extant"|"spec"|undefined} environment + * @memberof flyteidl.core.ExecutionEnv + * @instance + */ + Object.defineProperty(ExecutionEnv.prototype, "environment", { + get: $util.oneOfGetter($oneOfFields = ["extant", "spec"]), + set: $util.oneOfSetter($oneOfFields) + }); + + /** + * Creates a new ExecutionEnv instance using the specified properties. + * @function create + * @memberof flyteidl.core.ExecutionEnv + * @static + * @param {flyteidl.core.IExecutionEnv=} [properties] Properties to set + * @returns {flyteidl.core.ExecutionEnv} ExecutionEnv instance + */ + ExecutionEnv.create = function create(properties) { + return new ExecutionEnv(properties); + }; + + /** + * Encodes the specified ExecutionEnv message. Does not implicitly {@link flyteidl.core.ExecutionEnv.verify|verify} messages. + * @function encode + * @memberof flyteidl.core.ExecutionEnv + * @static + * @param {flyteidl.core.IExecutionEnv} message ExecutionEnv message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ExecutionEnv.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && message.hasOwnProperty("id")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.id); + if (message.type != null && message.hasOwnProperty("type")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.type); + if (message.extant != null && message.hasOwnProperty("extant")) + $root.google.protobuf.Struct.encode(message.extant, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.spec != null && message.hasOwnProperty("spec")) + $root.google.protobuf.Struct.encode(message.spec, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); + return writer; + }; + + /** + * Decodes an ExecutionEnv message from the specified reader or buffer. + * @function decode + * @memberof flyteidl.core.ExecutionEnv + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {flyteidl.core.ExecutionEnv} ExecutionEnv + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ExecutionEnv.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + var end = length === undefined ? reader.len : reader.pos + length, message = new $root.flyteidl.core.ExecutionEnv(); + while (reader.pos < end) { + var tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + message.id = reader.string(); + break; + case 2: + message.type = reader.string(); + break; + case 3: + message.extant = $root.google.protobuf.Struct.decode(reader, reader.uint32()); + break; + case 4: + message.spec = $root.google.protobuf.Struct.decode(reader, reader.uint32()); + break; + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Verifies an ExecutionEnv message. + * @function verify + * @memberof flyteidl.core.ExecutionEnv + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ExecutionEnv.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + var properties = {}; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isString(message.id)) + return "id: string expected"; + if (message.type != null && message.hasOwnProperty("type")) + if (!$util.isString(message.type)) + return "type: string expected"; + if (message.extant != null && message.hasOwnProperty("extant")) { + properties.environment = 1; + { + var error = $root.google.protobuf.Struct.verify(message.extant); + if (error) + return "extant." + error; + } + } + if (message.spec != null && message.hasOwnProperty("spec")) { + if (properties.environment === 1) + return "environment: multiple values"; + properties.environment = 1; + { + var error = $root.google.protobuf.Struct.verify(message.spec); + if (error) + return "spec." + error; + } + } + return null; + }; + + return ExecutionEnv; + })(); + core.Span = (function() { /** @@ -33613,6 +33955,7 @@ * @property {boolean|null} [overwriteCache] ExecutionSpec overwriteCache * @property {flyteidl.admin.IEnvs|null} [envs] ExecutionSpec envs * @property {Array.|null} [tags] ExecutionSpec tags + * @property {Array.|null} [executionEnvAssignments] ExecutionSpec executionEnvAssignments */ /** @@ -33625,6 +33968,7 @@ */ function ExecutionSpec(properties) { this.tags = []; + this.executionEnvAssignments = []; if (properties) for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -33767,6 +34111,14 @@ */ ExecutionSpec.prototype.tags = $util.emptyArray; + /** + * ExecutionSpec executionEnvAssignments. + * @member {Array.} executionEnvAssignments + * @memberof flyteidl.admin.ExecutionSpec + * @instance + */ + ExecutionSpec.prototype.executionEnvAssignments = $util.emptyArray; + // OneOf field names bound to virtual getters and setters var $oneOfFields; @@ -33840,6 +34192,9 @@ if (message.tags != null && message.tags.length) for (var i = 0; i < message.tags.length; ++i) writer.uint32(/* id 24, wireType 2 =*/194).string(message.tags[i]); + if (message.executionEnvAssignments != null && message.executionEnvAssignments.length) + for (var i = 0; i < message.executionEnvAssignments.length; ++i) + $root.flyteidl.core.ExecutionEnvAssignment.encode(message.executionEnvAssignments[i], writer.uint32(/* id 25, wireType 2 =*/202).fork()).ldelim(); return writer; }; @@ -33914,6 +34269,11 @@ message.tags = []; message.tags.push(reader.string()); break; + case 25: + if (!(message.executionEnvAssignments && message.executionEnvAssignments.length)) + message.executionEnvAssignments = []; + message.executionEnvAssignments.push($root.flyteidl.core.ExecutionEnvAssignment.decode(reader, reader.uint32())); + break; default: reader.skipType(tag & 7); break; @@ -34022,6 +34382,15 @@ if (!$util.isString(message.tags[i])) return "tags: string[] expected"; } + if (message.executionEnvAssignments != null && message.hasOwnProperty("executionEnvAssignments")) { + if (!Array.isArray(message.executionEnvAssignments)) + return "executionEnvAssignments: array expected"; + for (var i = 0; i < message.executionEnvAssignments.length; ++i) { + var error = $root.flyteidl.core.ExecutionEnvAssignment.verify(message.executionEnvAssignments[i]); + if (error) + return "executionEnvAssignments." + error; + } + } return null; }; @@ -36513,6 +36882,7 @@ * @property {google.protobuf.IBoolValue|null} [interruptible] LaunchPlanSpec interruptible * @property {boolean|null} [overwriteCache] LaunchPlanSpec overwriteCache * @property {flyteidl.admin.IEnvs|null} [envs] LaunchPlanSpec envs + * @property {Array.|null} [executionEnvAssignments] LaunchPlanSpec executionEnvAssignments */ /** @@ -36524,6 +36894,7 @@ * @param {flyteidl.admin.ILaunchPlanSpec=} [properties] Properties to set */ function LaunchPlanSpec(properties) { + this.executionEnvAssignments = []; if (properties) for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -36658,6 +37029,14 @@ */ LaunchPlanSpec.prototype.envs = null; + /** + * LaunchPlanSpec executionEnvAssignments. + * @member {Array.} executionEnvAssignments + * @memberof flyteidl.admin.LaunchPlanSpec + * @instance + */ + LaunchPlanSpec.prototype.executionEnvAssignments = $util.emptyArray; + /** * Creates a new LaunchPlanSpec instance using the specified properties. * @function create @@ -36714,6 +37093,9 @@ writer.uint32(/* id 20, wireType 0 =*/160).bool(message.overwriteCache); if (message.envs != null && message.hasOwnProperty("envs")) $root.flyteidl.admin.Envs.encode(message.envs, writer.uint32(/* id 21, wireType 2 =*/170).fork()).ldelim(); + if (message.executionEnvAssignments != null && message.executionEnvAssignments.length) + for (var i = 0; i < message.executionEnvAssignments.length; ++i) + $root.flyteidl.core.ExecutionEnvAssignment.encode(message.executionEnvAssignments[i], writer.uint32(/* id 22, wireType 2 =*/178).fork()).ldelim(); return writer; }; @@ -36783,6 +37165,11 @@ case 21: message.envs = $root.flyteidl.admin.Envs.decode(reader, reader.uint32()); break; + case 22: + if (!(message.executionEnvAssignments && message.executionEnvAssignments.length)) + message.executionEnvAssignments = []; + message.executionEnvAssignments.push($root.flyteidl.core.ExecutionEnvAssignment.decode(reader, reader.uint32())); + break; default: reader.skipType(tag & 7); break; @@ -36876,6 +37263,15 @@ if (error) return "envs." + error; } + if (message.executionEnvAssignments != null && message.hasOwnProperty("executionEnvAssignments")) { + if (!Array.isArray(message.executionEnvAssignments)) + return "executionEnvAssignments: array expected"; + for (var i = 0; i < message.executionEnvAssignments.length; ++i) { + var error = $root.flyteidl.core.ExecutionEnvAssignment.verify(message.executionEnvAssignments[i]); + if (error) + return "executionEnvAssignments." + error; + } + } return null; }; @@ -39225,6 +39621,7 @@ * @property {google.protobuf.IBoolValue|null} [interruptible] WorkflowExecutionConfig interruptible * @property {boolean|null} [overwriteCache] WorkflowExecutionConfig overwriteCache * @property {flyteidl.admin.IEnvs|null} [envs] WorkflowExecutionConfig envs + * @property {Array.|null} [executionEnvAssignments] WorkflowExecutionConfig executionEnvAssignments */ /** @@ -39236,6 +39633,7 @@ * @param {flyteidl.admin.IWorkflowExecutionConfig=} [properties] Properties to set */ function WorkflowExecutionConfig(properties) { + this.executionEnvAssignments = []; if (properties) for (var keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -39306,6 +39704,14 @@ */ WorkflowExecutionConfig.prototype.envs = null; + /** + * WorkflowExecutionConfig executionEnvAssignments. + * @member {Array.} executionEnvAssignments + * @memberof flyteidl.admin.WorkflowExecutionConfig + * @instance + */ + WorkflowExecutionConfig.prototype.executionEnvAssignments = $util.emptyArray; + /** * Creates a new WorkflowExecutionConfig instance using the specified properties. * @function create @@ -39346,6 +39752,9 @@ writer.uint32(/* id 7, wireType 0 =*/56).bool(message.overwriteCache); if (message.envs != null && message.hasOwnProperty("envs")) $root.flyteidl.admin.Envs.encode(message.envs, writer.uint32(/* id 8, wireType 2 =*/66).fork()).ldelim(); + if (message.executionEnvAssignments != null && message.executionEnvAssignments.length) + for (var i = 0; i < message.executionEnvAssignments.length; ++i) + $root.flyteidl.core.ExecutionEnvAssignment.encode(message.executionEnvAssignments[i], writer.uint32(/* id 9, wireType 2 =*/74).fork()).ldelim(); return writer; }; @@ -39391,6 +39800,11 @@ case 8: message.envs = $root.flyteidl.admin.Envs.decode(reader, reader.uint32()); break; + case 9: + if (!(message.executionEnvAssignments && message.executionEnvAssignments.length)) + message.executionEnvAssignments = []; + message.executionEnvAssignments.push($root.flyteidl.core.ExecutionEnvAssignment.decode(reader, reader.uint32())); + break; default: reader.skipType(tag & 7); break; @@ -39446,6 +39860,15 @@ if (error) return "envs." + error; } + if (message.executionEnvAssignments != null && message.hasOwnProperty("executionEnvAssignments")) { + if (!Array.isArray(message.executionEnvAssignments)) + return "executionEnvAssignments: array expected"; + for (var i = 0; i < message.executionEnvAssignments.length; ++i) { + var error = $root.flyteidl.core.ExecutionEnvAssignment.verify(message.executionEnvAssignments[i]); + if (error) + return "executionEnvAssignments." + error; + } + } return null; }; diff --git a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py index bdcda37744..ed584320af 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.py @@ -15,6 +15,7 @@ from flyteidl.admin import common_pb2 as flyteidl_dot_admin_dot_common__pb2 from flyteidl.core import literals_pb2 as flyteidl_dot_core_dot_literals__pb2 from flyteidl.core import execution_pb2 as flyteidl_dot_core_dot_execution__pb2 +from flyteidl.core import execution_envs_pb2 as flyteidl_dot_core_dot_execution__envs__pb2 from flyteidl.core import artifact_id_pb2 as flyteidl_dot_core_dot_artifact__id__pb2 from flyteidl.core import identifier_pb2 as flyteidl_dot_core_dot_identifier__pb2 from flyteidl.core import metrics_pb2 as flyteidl_dot_core_dot_metrics__pb2 @@ -24,7 +25,7 @@ from google.protobuf import wrappers_pb2 as google_dot_protobuf_dot_wrappers__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1e\x66lyteidl/admin/execution.proto\x12\x0e\x66lyteidl.admin\x1a\'flyteidl/admin/cluster_assignment.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\x1f\x66lyteidl/core/artifact_id.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1b\x66lyteidl/core/metrics.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\"\xd6\x01\n\x16\x45xecutionCreateRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\x31\n\x04spec\x18\x04 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12\x31\n\x06inputs\x18\x05 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x06inputs\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org\"\x99\x01\n\x18\x45xecutionRelaunchRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\'\n\x0foverwrite_cache\x18\x04 \x01(\x08R\x0eoverwriteCacheJ\x04\x08\x02\x10\x03\"\xa8\x01\n\x17\x45xecutionRecoverRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\"U\n\x17\x45xecutionCreateResponse\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"Y\n\x1bWorkflowExecutionGetRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\xb6\x01\n\tExecution\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x31\n\x04spec\x18\x02 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12:\n\x07\x63losure\x18\x03 \x01(\x0b\x32 .flyteidl.admin.ExecutionClosureR\x07\x63losure\"`\n\rExecutionList\x12\x39\n\nexecutions\x18\x01 \x03(\x0b\x32\x19.flyteidl.admin.ExecutionR\nexecutions\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"e\n\x0eLiteralMapBlob\x12\x37\n\x06values\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\x06values\x12\x12\n\x03uri\x18\x02 \x01(\tH\x00R\x03uriB\x06\n\x04\x64\x61ta\"C\n\rAbortMetadata\x12\x14\n\x05\x63\x61use\x18\x01 \x01(\tR\x05\x63\x61use\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\"\x98\x07\n\x10\x45xecutionClosure\x12>\n\x07outputs\x18\x01 \x01(\x0b\x32\x1e.flyteidl.admin.LiteralMapBlobB\x02\x18\x01H\x00R\x07outputs\x12\x35\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x1d.flyteidl.core.ExecutionErrorH\x00R\x05\x65rror\x12%\n\x0b\x61\x62ort_cause\x18\n \x01(\tB\x02\x18\x01H\x00R\nabortCause\x12\x46\n\x0e\x61\x62ort_metadata\x18\x0c \x01(\x0b\x32\x1d.flyteidl.admin.AbortMetadataH\x00R\rabortMetadata\x12@\n\x0boutput_data\x18\r \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\noutputData\x12\x46\n\x0f\x63omputed_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x0e\x63omputedInputs\x12<\n\x05phase\x18\x04 \x01(\x0e\x32&.flyteidl.core.WorkflowExecution.PhaseR\x05phase\x12\x39\n\nstarted_at\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tstartedAt\x12\x35\n\x08\x64uration\x18\x06 \x01(\x0b\x32\x19.google.protobuf.DurationR\x08\x64uration\x12\x39\n\ncreated_at\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\x12\x42\n\rnotifications\x18\t \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\x12:\n\x0bworkflow_id\x18\x0b \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nworkflowId\x12]\n\x14state_change_details\x18\x0e \x01(\x0b\x32+.flyteidl.admin.ExecutionStateChangeDetailsR\x12stateChangeDetailsB\x0f\n\routput_result\"[\n\x0eSystemMetadata\x12+\n\x11\x65xecution_cluster\x18\x01 \x01(\tR\x10\x65xecutionCluster\x12\x1c\n\tnamespace\x18\x02 \x01(\tR\tnamespace\"\x85\x05\n\x11\x45xecutionMetadata\x12\x43\n\x04mode\x18\x01 \x01(\x0e\x32/.flyteidl.admin.ExecutionMetadata.ExecutionModeR\x04mode\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\x12\x18\n\x07nesting\x18\x03 \x01(\rR\x07nesting\x12=\n\x0cscheduled_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\x0bscheduledAt\x12Z\n\x15parent_node_execution\x18\x05 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x13parentNodeExecution\x12[\n\x13reference_execution\x18\x10 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x12referenceExecution\x12G\n\x0fsystem_metadata\x18\x11 \x01(\x0b\x32\x1e.flyteidl.admin.SystemMetadataR\x0esystemMetadata\x12<\n\x0c\x61rtifact_ids\x18\x12 \x03(\x0b\x32\x19.flyteidl.core.ArtifactIDR\x0b\x61rtifactIds\"t\n\rExecutionMode\x12\n\n\x06MANUAL\x10\x00\x12\r\n\tSCHEDULED\x10\x01\x12\n\n\x06SYSTEM\x10\x02\x12\x0c\n\x08RELAUNCH\x10\x03\x12\x12\n\x0e\x43HILD_WORKFLOW\x10\x04\x12\r\n\tRECOVERED\x10\x05\x12\x0b\n\x07TRIGGER\x10\x06\"V\n\x10NotificationList\x12\x42\n\rnotifications\x18\x01 \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\"\x90\x08\n\rExecutionSpec\x12:\n\x0blaunch_plan\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nlaunchPlan\x12\x35\n\x06inputs\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x06inputs\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\x12H\n\rnotifications\x18\x05 \x01(\x0b\x32 .flyteidl.admin.NotificationListH\x00R\rnotifications\x12!\n\x0b\x64isable_all\x18\x06 \x01(\x08H\x00R\ndisableAll\x12.\n\x06labels\x18\x07 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x08 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12I\n\x10security_context\x18\n \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12\x39\n\tauth_role\x18\x10 \x01(\x0b\x32\x18.flyteidl.admin.AuthRoleB\x02\x18\x01R\x08\x61uthRole\x12M\n\x12quality_of_service\x18\x11 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceR\x10qualityOfService\x12\'\n\x0fmax_parallelism\x18\x12 \x01(\x05R\x0emaxParallelism\x12X\n\x16raw_output_data_config\x18\x13 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12P\n\x12\x63luster_assignment\x18\x14 \x01(\x0b\x32!.flyteidl.admin.ClusterAssignmentR\x11\x63lusterAssignment\x12@\n\rinterruptible\x18\x15 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x16 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x17 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\x12\x12\n\x04tags\x18\x18 \x03(\tR\x04tagsB\x18\n\x16notification_overridesJ\x04\x08\x04\x10\x05\"m\n\x19\x45xecutionTerminateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x63\x61use\x18\x02 \x01(\tR\x05\x63\x61use\"\x1c\n\x1a\x45xecutionTerminateResponse\"]\n\x1fWorkflowExecutionGetDataRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\x88\x02\n WorkflowExecutionGetDataResponse\x12\x35\n\x07outputs\x18\x01 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x07outputs\x12\x33\n\x06inputs\x18\x02 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x06inputs\x12:\n\x0b\x66ull_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\nfullInputs\x12<\n\x0c\x66ull_outputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ullOutputs\"\x8a\x01\n\x16\x45xecutionUpdateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x34\n\x05state\x18\x02 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\"\xae\x01\n\x1b\x45xecutionStateChangeDetails\x12\x34\n\x05state\x18\x01 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\x12;\n\x0boccurred_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\noccurredAt\x12\x1c\n\tprincipal\x18\x03 \x01(\tR\tprincipal\"\x19\n\x17\x45xecutionUpdateResponse\"v\n\"WorkflowExecutionGetMetricsRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x64\x65pth\x18\x02 \x01(\x05R\x05\x64\x65pth\"N\n#WorkflowExecutionGetMetricsResponse\x12\'\n\x04span\x18\x01 \x01(\x0b\x32\x13.flyteidl.core.SpanR\x04span\"y\n\x19\x45xecutionCountsGetRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x10\n\x03org\x18\x03 \x01(\tR\x03org\x12\x18\n\x07\x66ilters\x18\x04 \x01(\tR\x07\x66ilters\"l\n\x16\x45xecutionCountsByPhase\x12<\n\x05phase\x18\x01 \x01(\x0e\x32&.flyteidl.core.WorkflowExecution.PhaseR\x05phase\x12\x14\n\x05\x63ount\x18\x02 \x01(\x03R\x05\x63ount\"o\n\x1a\x45xecutionCountsGetResponse\x12Q\n\x10\x65xecution_counts\x18\x01 \x03(\x0b\x32&.flyteidl.admin.ExecutionCountsByPhaseR\x0f\x65xecutionCounts\"f\n RunningExecutionsCountGetRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x10\n\x03org\x18\x03 \x01(\tR\x03org\"9\n!RunningExecutionsCountGetResponse\x12\x14\n\x05\x63ount\x18\x01 \x01(\x03R\x05\x63ount*>\n\x0e\x45xecutionState\x12\x14\n\x10\x45XECUTION_ACTIVE\x10\x00\x12\x16\n\x12\x45XECUTION_ARCHIVED\x10\x01\x42\xba\x01\n\x12\x63om.flyteidl.adminB\x0e\x45xecutionProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1e\x66lyteidl/admin/execution.proto\x12\x0e\x66lyteidl.admin\x1a\'flyteidl/admin/cluster_assignment.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\"flyteidl/core/execution_envs.proto\x1a\x1f\x66lyteidl/core/artifact_id.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1b\x66lyteidl/core/metrics.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\"\xd6\x01\n\x16\x45xecutionCreateRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\x31\n\x04spec\x18\x04 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12\x31\n\x06inputs\x18\x05 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x06inputs\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org\"\x99\x01\n\x18\x45xecutionRelaunchRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x03 \x01(\tR\x04name\x12\'\n\x0foverwrite_cache\x18\x04 \x01(\x08R\x0eoverwriteCacheJ\x04\x08\x02\x10\x03\"\xa8\x01\n\x17\x45xecutionRecoverRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\"U\n\x17\x45xecutionCreateResponse\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"Y\n\x1bWorkflowExecutionGetRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\xb6\x01\n\tExecution\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x31\n\x04spec\x18\x02 \x01(\x0b\x32\x1d.flyteidl.admin.ExecutionSpecR\x04spec\x12:\n\x07\x63losure\x18\x03 \x01(\x0b\x32 .flyteidl.admin.ExecutionClosureR\x07\x63losure\"`\n\rExecutionList\x12\x39\n\nexecutions\x18\x01 \x03(\x0b\x32\x19.flyteidl.admin.ExecutionR\nexecutions\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"e\n\x0eLiteralMapBlob\x12\x37\n\x06values\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\x06values\x12\x12\n\x03uri\x18\x02 \x01(\tH\x00R\x03uriB\x06\n\x04\x64\x61ta\"C\n\rAbortMetadata\x12\x14\n\x05\x63\x61use\x18\x01 \x01(\tR\x05\x63\x61use\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\"\x98\x07\n\x10\x45xecutionClosure\x12>\n\x07outputs\x18\x01 \x01(\x0b\x32\x1e.flyteidl.admin.LiteralMapBlobB\x02\x18\x01H\x00R\x07outputs\x12\x35\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x1d.flyteidl.core.ExecutionErrorH\x00R\x05\x65rror\x12%\n\x0b\x61\x62ort_cause\x18\n \x01(\tB\x02\x18\x01H\x00R\nabortCause\x12\x46\n\x0e\x61\x62ort_metadata\x18\x0c \x01(\x0b\x32\x1d.flyteidl.admin.AbortMetadataH\x00R\rabortMetadata\x12@\n\x0boutput_data\x18\r \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01H\x00R\noutputData\x12\x46\n\x0f\x63omputed_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x0e\x63omputedInputs\x12<\n\x05phase\x18\x04 \x01(\x0e\x32&.flyteidl.core.WorkflowExecution.PhaseR\x05phase\x12\x39\n\nstarted_at\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tstartedAt\x12\x35\n\x08\x64uration\x18\x06 \x01(\x0b\x32\x19.google.protobuf.DurationR\x08\x64uration\x12\x39\n\ncreated_at\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x08 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\x12\x42\n\rnotifications\x18\t \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\x12:\n\x0bworkflow_id\x18\x0b \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nworkflowId\x12]\n\x14state_change_details\x18\x0e \x01(\x0b\x32+.flyteidl.admin.ExecutionStateChangeDetailsR\x12stateChangeDetailsB\x0f\n\routput_result\"[\n\x0eSystemMetadata\x12+\n\x11\x65xecution_cluster\x18\x01 \x01(\tR\x10\x65xecutionCluster\x12\x1c\n\tnamespace\x18\x02 \x01(\tR\tnamespace\"\x85\x05\n\x11\x45xecutionMetadata\x12\x43\n\x04mode\x18\x01 \x01(\x0e\x32/.flyteidl.admin.ExecutionMetadata.ExecutionModeR\x04mode\x12\x1c\n\tprincipal\x18\x02 \x01(\tR\tprincipal\x12\x18\n\x07nesting\x18\x03 \x01(\rR\x07nesting\x12=\n\x0cscheduled_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\x0bscheduledAt\x12Z\n\x15parent_node_execution\x18\x05 \x01(\x0b\x32&.flyteidl.core.NodeExecutionIdentifierR\x13parentNodeExecution\x12[\n\x13reference_execution\x18\x10 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x12referenceExecution\x12G\n\x0fsystem_metadata\x18\x11 \x01(\x0b\x32\x1e.flyteidl.admin.SystemMetadataR\x0esystemMetadata\x12<\n\x0c\x61rtifact_ids\x18\x12 \x03(\x0b\x32\x19.flyteidl.core.ArtifactIDR\x0b\x61rtifactIds\"t\n\rExecutionMode\x12\n\n\x06MANUAL\x10\x00\x12\r\n\tSCHEDULED\x10\x01\x12\n\n\x06SYSTEM\x10\x02\x12\x0c\n\x08RELAUNCH\x10\x03\x12\x12\n\x0e\x43HILD_WORKFLOW\x10\x04\x12\r\n\tRECOVERED\x10\x05\x12\x0b\n\x07TRIGGER\x10\x06\"V\n\x10NotificationList\x12\x42\n\rnotifications\x18\x01 \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\"\xf3\x08\n\rExecutionSpec\x12:\n\x0blaunch_plan\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nlaunchPlan\x12\x35\n\x06inputs\x18\x02 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapB\x02\x18\x01R\x06inputs\x12=\n\x08metadata\x18\x03 \x01(\x0b\x32!.flyteidl.admin.ExecutionMetadataR\x08metadata\x12H\n\rnotifications\x18\x05 \x01(\x0b\x32 .flyteidl.admin.NotificationListH\x00R\rnotifications\x12!\n\x0b\x64isable_all\x18\x06 \x01(\x08H\x00R\ndisableAll\x12.\n\x06labels\x18\x07 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x08 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12I\n\x10security_context\x18\n \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12\x39\n\tauth_role\x18\x10 \x01(\x0b\x32\x18.flyteidl.admin.AuthRoleB\x02\x18\x01R\x08\x61uthRole\x12M\n\x12quality_of_service\x18\x11 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceR\x10qualityOfService\x12\'\n\x0fmax_parallelism\x18\x12 \x01(\x05R\x0emaxParallelism\x12X\n\x16raw_output_data_config\x18\x13 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12P\n\x12\x63luster_assignment\x18\x14 \x01(\x0b\x32!.flyteidl.admin.ClusterAssignmentR\x11\x63lusterAssignment\x12@\n\rinterruptible\x18\x15 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x16 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x17 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\x12\x12\n\x04tags\x18\x18 \x03(\tR\x04tags\x12\x61\n\x19\x65xecution_env_assignments\x18\x19 \x03(\x0b\x32%.flyteidl.core.ExecutionEnvAssignmentR\x17\x65xecutionEnvAssignmentsB\x18\n\x16notification_overridesJ\x04\x08\x04\x10\x05\"m\n\x19\x45xecutionTerminateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x63\x61use\x18\x02 \x01(\tR\x05\x63\x61use\"\x1c\n\x1a\x45xecutionTerminateResponse\"]\n\x1fWorkflowExecutionGetDataRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\"\x88\x02\n WorkflowExecutionGetDataResponse\x12\x35\n\x07outputs\x18\x01 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x07outputs\x12\x33\n\x06inputs\x18\x02 \x01(\x0b\x32\x17.flyteidl.admin.UrlBlobB\x02\x18\x01R\x06inputs\x12:\n\x0b\x66ull_inputs\x18\x03 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\nfullInputs\x12<\n\x0c\x66ull_outputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ullOutputs\"\x8a\x01\n\x16\x45xecutionUpdateRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x34\n\x05state\x18\x02 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\"\xae\x01\n\x1b\x45xecutionStateChangeDetails\x12\x34\n\x05state\x18\x01 \x01(\x0e\x32\x1e.flyteidl.admin.ExecutionStateR\x05state\x12;\n\x0boccurred_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\noccurredAt\x12\x1c\n\tprincipal\x18\x03 \x01(\tR\tprincipal\"\x19\n\x17\x45xecutionUpdateResponse\"v\n\"WorkflowExecutionGetMetricsRequest\x12:\n\x02id\x18\x01 \x01(\x0b\x32*.flyteidl.core.WorkflowExecutionIdentifierR\x02id\x12\x14\n\x05\x64\x65pth\x18\x02 \x01(\x05R\x05\x64\x65pth\"N\n#WorkflowExecutionGetMetricsResponse\x12\'\n\x04span\x18\x01 \x01(\x0b\x32\x13.flyteidl.core.SpanR\x04span\"y\n\x19\x45xecutionCountsGetRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x10\n\x03org\x18\x03 \x01(\tR\x03org\x12\x18\n\x07\x66ilters\x18\x04 \x01(\tR\x07\x66ilters\"l\n\x16\x45xecutionCountsByPhase\x12<\n\x05phase\x18\x01 \x01(\x0e\x32&.flyteidl.core.WorkflowExecution.PhaseR\x05phase\x12\x14\n\x05\x63ount\x18\x02 \x01(\x03R\x05\x63ount\"o\n\x1a\x45xecutionCountsGetResponse\x12Q\n\x10\x65xecution_counts\x18\x01 \x03(\x0b\x32&.flyteidl.admin.ExecutionCountsByPhaseR\x0f\x65xecutionCounts\"f\n RunningExecutionsCountGetRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x10\n\x03org\x18\x03 \x01(\tR\x03org\"9\n!RunningExecutionsCountGetResponse\x12\x14\n\x05\x63ount\x18\x01 \x01(\x03R\x05\x63ount*>\n\x0e\x45xecutionState\x12\x14\n\x10\x45XECUTION_ACTIVE\x10\x00\x12\x16\n\x12\x45XECUTION_ARCHIVED\x10\x01\x42\xba\x01\n\x12\x63om.flyteidl.adminB\x0e\x45xecutionProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -51,64 +52,64 @@ _WORKFLOWEXECUTIONGETDATARESPONSE.fields_by_name['outputs']._serialized_options = b'\030\001' _WORKFLOWEXECUTIONGETDATARESPONSE.fields_by_name['inputs']._options = None _WORKFLOWEXECUTIONGETDATARESPONSE.fields_by_name['inputs']._serialized_options = b'\030\001' - _globals['_EXECUTIONSTATE']._serialized_start=5931 - _globals['_EXECUTIONSTATE']._serialized_end=5993 - _globals['_EXECUTIONCREATEREQUEST']._serialized_start=403 - _globals['_EXECUTIONCREATEREQUEST']._serialized_end=617 - _globals['_EXECUTIONRELAUNCHREQUEST']._serialized_start=620 - _globals['_EXECUTIONRELAUNCHREQUEST']._serialized_end=773 - _globals['_EXECUTIONRECOVERREQUEST']._serialized_start=776 - _globals['_EXECUTIONRECOVERREQUEST']._serialized_end=944 - _globals['_EXECUTIONCREATERESPONSE']._serialized_start=946 - _globals['_EXECUTIONCREATERESPONSE']._serialized_end=1031 - _globals['_WORKFLOWEXECUTIONGETREQUEST']._serialized_start=1033 - _globals['_WORKFLOWEXECUTIONGETREQUEST']._serialized_end=1122 - _globals['_EXECUTION']._serialized_start=1125 - _globals['_EXECUTION']._serialized_end=1307 - _globals['_EXECUTIONLIST']._serialized_start=1309 - _globals['_EXECUTIONLIST']._serialized_end=1405 - _globals['_LITERALMAPBLOB']._serialized_start=1407 - _globals['_LITERALMAPBLOB']._serialized_end=1508 - _globals['_ABORTMETADATA']._serialized_start=1510 - _globals['_ABORTMETADATA']._serialized_end=1577 - _globals['_EXECUTIONCLOSURE']._serialized_start=1580 - _globals['_EXECUTIONCLOSURE']._serialized_end=2500 - _globals['_SYSTEMMETADATA']._serialized_start=2502 - _globals['_SYSTEMMETADATA']._serialized_end=2593 - _globals['_EXECUTIONMETADATA']._serialized_start=2596 - _globals['_EXECUTIONMETADATA']._serialized_end=3241 - _globals['_EXECUTIONMETADATA_EXECUTIONMODE']._serialized_start=3125 - _globals['_EXECUTIONMETADATA_EXECUTIONMODE']._serialized_end=3241 - _globals['_NOTIFICATIONLIST']._serialized_start=3243 - _globals['_NOTIFICATIONLIST']._serialized_end=3329 - _globals['_EXECUTIONSPEC']._serialized_start=3332 - _globals['_EXECUTIONSPEC']._serialized_end=4372 - _globals['_EXECUTIONTERMINATEREQUEST']._serialized_start=4374 - _globals['_EXECUTIONTERMINATEREQUEST']._serialized_end=4483 - _globals['_EXECUTIONTERMINATERESPONSE']._serialized_start=4485 - _globals['_EXECUTIONTERMINATERESPONSE']._serialized_end=4513 - _globals['_WORKFLOWEXECUTIONGETDATAREQUEST']._serialized_start=4515 - _globals['_WORKFLOWEXECUTIONGETDATAREQUEST']._serialized_end=4608 - _globals['_WORKFLOWEXECUTIONGETDATARESPONSE']._serialized_start=4611 - _globals['_WORKFLOWEXECUTIONGETDATARESPONSE']._serialized_end=4875 - _globals['_EXECUTIONUPDATEREQUEST']._serialized_start=4878 - _globals['_EXECUTIONUPDATEREQUEST']._serialized_end=5016 - _globals['_EXECUTIONSTATECHANGEDETAILS']._serialized_start=5019 - _globals['_EXECUTIONSTATECHANGEDETAILS']._serialized_end=5193 - _globals['_EXECUTIONUPDATERESPONSE']._serialized_start=5195 - _globals['_EXECUTIONUPDATERESPONSE']._serialized_end=5220 - _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_start=5222 - _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_end=5340 - _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_start=5342 - _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_end=5420 - _globals['_EXECUTIONCOUNTSGETREQUEST']._serialized_start=5422 - _globals['_EXECUTIONCOUNTSGETREQUEST']._serialized_end=5543 - _globals['_EXECUTIONCOUNTSBYPHASE']._serialized_start=5545 - _globals['_EXECUTIONCOUNTSBYPHASE']._serialized_end=5653 - _globals['_EXECUTIONCOUNTSGETRESPONSE']._serialized_start=5655 - _globals['_EXECUTIONCOUNTSGETRESPONSE']._serialized_end=5766 - _globals['_RUNNINGEXECUTIONSCOUNTGETREQUEST']._serialized_start=5768 - _globals['_RUNNINGEXECUTIONSCOUNTGETREQUEST']._serialized_end=5870 - _globals['_RUNNINGEXECUTIONSCOUNTGETRESPONSE']._serialized_start=5872 - _globals['_RUNNINGEXECUTIONSCOUNTGETRESPONSE']._serialized_end=5929 + _globals['_EXECUTIONSTATE']._serialized_start=6066 + _globals['_EXECUTIONSTATE']._serialized_end=6128 + _globals['_EXECUTIONCREATEREQUEST']._serialized_start=439 + _globals['_EXECUTIONCREATEREQUEST']._serialized_end=653 + _globals['_EXECUTIONRELAUNCHREQUEST']._serialized_start=656 + _globals['_EXECUTIONRELAUNCHREQUEST']._serialized_end=809 + _globals['_EXECUTIONRECOVERREQUEST']._serialized_start=812 + _globals['_EXECUTIONRECOVERREQUEST']._serialized_end=980 + _globals['_EXECUTIONCREATERESPONSE']._serialized_start=982 + _globals['_EXECUTIONCREATERESPONSE']._serialized_end=1067 + _globals['_WORKFLOWEXECUTIONGETREQUEST']._serialized_start=1069 + _globals['_WORKFLOWEXECUTIONGETREQUEST']._serialized_end=1158 + _globals['_EXECUTION']._serialized_start=1161 + _globals['_EXECUTION']._serialized_end=1343 + _globals['_EXECUTIONLIST']._serialized_start=1345 + _globals['_EXECUTIONLIST']._serialized_end=1441 + _globals['_LITERALMAPBLOB']._serialized_start=1443 + _globals['_LITERALMAPBLOB']._serialized_end=1544 + _globals['_ABORTMETADATA']._serialized_start=1546 + _globals['_ABORTMETADATA']._serialized_end=1613 + _globals['_EXECUTIONCLOSURE']._serialized_start=1616 + _globals['_EXECUTIONCLOSURE']._serialized_end=2536 + _globals['_SYSTEMMETADATA']._serialized_start=2538 + _globals['_SYSTEMMETADATA']._serialized_end=2629 + _globals['_EXECUTIONMETADATA']._serialized_start=2632 + _globals['_EXECUTIONMETADATA']._serialized_end=3277 + _globals['_EXECUTIONMETADATA_EXECUTIONMODE']._serialized_start=3161 + _globals['_EXECUTIONMETADATA_EXECUTIONMODE']._serialized_end=3277 + _globals['_NOTIFICATIONLIST']._serialized_start=3279 + _globals['_NOTIFICATIONLIST']._serialized_end=3365 + _globals['_EXECUTIONSPEC']._serialized_start=3368 + _globals['_EXECUTIONSPEC']._serialized_end=4507 + _globals['_EXECUTIONTERMINATEREQUEST']._serialized_start=4509 + _globals['_EXECUTIONTERMINATEREQUEST']._serialized_end=4618 + _globals['_EXECUTIONTERMINATERESPONSE']._serialized_start=4620 + _globals['_EXECUTIONTERMINATERESPONSE']._serialized_end=4648 + _globals['_WORKFLOWEXECUTIONGETDATAREQUEST']._serialized_start=4650 + _globals['_WORKFLOWEXECUTIONGETDATAREQUEST']._serialized_end=4743 + _globals['_WORKFLOWEXECUTIONGETDATARESPONSE']._serialized_start=4746 + _globals['_WORKFLOWEXECUTIONGETDATARESPONSE']._serialized_end=5010 + _globals['_EXECUTIONUPDATEREQUEST']._serialized_start=5013 + _globals['_EXECUTIONUPDATEREQUEST']._serialized_end=5151 + _globals['_EXECUTIONSTATECHANGEDETAILS']._serialized_start=5154 + _globals['_EXECUTIONSTATECHANGEDETAILS']._serialized_end=5328 + _globals['_EXECUTIONUPDATERESPONSE']._serialized_start=5330 + _globals['_EXECUTIONUPDATERESPONSE']._serialized_end=5355 + _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_start=5357 + _globals['_WORKFLOWEXECUTIONGETMETRICSREQUEST']._serialized_end=5475 + _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_start=5477 + _globals['_WORKFLOWEXECUTIONGETMETRICSRESPONSE']._serialized_end=5555 + _globals['_EXECUTIONCOUNTSGETREQUEST']._serialized_start=5557 + _globals['_EXECUTIONCOUNTSGETREQUEST']._serialized_end=5678 + _globals['_EXECUTIONCOUNTSBYPHASE']._serialized_start=5680 + _globals['_EXECUTIONCOUNTSBYPHASE']._serialized_end=5788 + _globals['_EXECUTIONCOUNTSGETRESPONSE']._serialized_start=5790 + _globals['_EXECUTIONCOUNTSGETRESPONSE']._serialized_end=5901 + _globals['_RUNNINGEXECUTIONSCOUNTGETREQUEST']._serialized_start=5903 + _globals['_RUNNINGEXECUTIONSCOUNTGETREQUEST']._serialized_end=6005 + _globals['_RUNNINGEXECUTIONSCOUNTGETRESPONSE']._serialized_start=6007 + _globals['_RUNNINGEXECUTIONSCOUNTGETRESPONSE']._serialized_end=6064 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi index f4099ac3d6..9d3ed50c05 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/admin/execution_pb2.pyi @@ -2,6 +2,7 @@ from flyteidl.admin import cluster_assignment_pb2 as _cluster_assignment_pb2 from flyteidl.admin import common_pb2 as _common_pb2 from flyteidl.core import literals_pb2 as _literals_pb2 from flyteidl.core import execution_pb2 as _execution_pb2 +from flyteidl.core import execution_envs_pb2 as _execution_envs_pb2 from flyteidl.core import artifact_id_pb2 as _artifact_id_pb2 from flyteidl.core import identifier_pb2 as _identifier_pb2 from flyteidl.core import metrics_pb2 as _metrics_pb2 @@ -189,7 +190,7 @@ class NotificationList(_message.Message): def __init__(self, notifications: _Optional[_Iterable[_Union[_common_pb2.Notification, _Mapping]]] = ...) -> None: ... class ExecutionSpec(_message.Message): - __slots__ = ["launch_plan", "inputs", "metadata", "notifications", "disable_all", "labels", "annotations", "security_context", "auth_role", "quality_of_service", "max_parallelism", "raw_output_data_config", "cluster_assignment", "interruptible", "overwrite_cache", "envs", "tags"] + __slots__ = ["launch_plan", "inputs", "metadata", "notifications", "disable_all", "labels", "annotations", "security_context", "auth_role", "quality_of_service", "max_parallelism", "raw_output_data_config", "cluster_assignment", "interruptible", "overwrite_cache", "envs", "tags", "execution_env_assignments"] LAUNCH_PLAN_FIELD_NUMBER: _ClassVar[int] INPUTS_FIELD_NUMBER: _ClassVar[int] METADATA_FIELD_NUMBER: _ClassVar[int] @@ -207,6 +208,7 @@ class ExecutionSpec(_message.Message): OVERWRITE_CACHE_FIELD_NUMBER: _ClassVar[int] ENVS_FIELD_NUMBER: _ClassVar[int] TAGS_FIELD_NUMBER: _ClassVar[int] + EXECUTION_ENV_ASSIGNMENTS_FIELD_NUMBER: _ClassVar[int] launch_plan: _identifier_pb2.Identifier inputs: _literals_pb2.LiteralMap metadata: ExecutionMetadata @@ -224,7 +226,8 @@ class ExecutionSpec(_message.Message): overwrite_cache: bool envs: _common_pb2.Envs tags: _containers.RepeatedScalarFieldContainer[str] - def __init__(self, launch_plan: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., inputs: _Optional[_Union[_literals_pb2.LiteralMap, _Mapping]] = ..., metadata: _Optional[_Union[ExecutionMetadata, _Mapping]] = ..., notifications: _Optional[_Union[NotificationList, _Mapping]] = ..., disable_all: bool = ..., labels: _Optional[_Union[_common_pb2.Labels, _Mapping]] = ..., annotations: _Optional[_Union[_common_pb2.Annotations, _Mapping]] = ..., security_context: _Optional[_Union[_security_pb2.SecurityContext, _Mapping]] = ..., auth_role: _Optional[_Union[_common_pb2.AuthRole, _Mapping]] = ..., quality_of_service: _Optional[_Union[_execution_pb2.QualityOfService, _Mapping]] = ..., max_parallelism: _Optional[int] = ..., raw_output_data_config: _Optional[_Union[_common_pb2.RawOutputDataConfig, _Mapping]] = ..., cluster_assignment: _Optional[_Union[_cluster_assignment_pb2.ClusterAssignment, _Mapping]] = ..., interruptible: _Optional[_Union[_wrappers_pb2.BoolValue, _Mapping]] = ..., overwrite_cache: bool = ..., envs: _Optional[_Union[_common_pb2.Envs, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ...) -> None: ... + execution_env_assignments: _containers.RepeatedCompositeFieldContainer[_execution_envs_pb2.ExecutionEnvAssignment] + def __init__(self, launch_plan: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., inputs: _Optional[_Union[_literals_pb2.LiteralMap, _Mapping]] = ..., metadata: _Optional[_Union[ExecutionMetadata, _Mapping]] = ..., notifications: _Optional[_Union[NotificationList, _Mapping]] = ..., disable_all: bool = ..., labels: _Optional[_Union[_common_pb2.Labels, _Mapping]] = ..., annotations: _Optional[_Union[_common_pb2.Annotations, _Mapping]] = ..., security_context: _Optional[_Union[_security_pb2.SecurityContext, _Mapping]] = ..., auth_role: _Optional[_Union[_common_pb2.AuthRole, _Mapping]] = ..., quality_of_service: _Optional[_Union[_execution_pb2.QualityOfService, _Mapping]] = ..., max_parallelism: _Optional[int] = ..., raw_output_data_config: _Optional[_Union[_common_pb2.RawOutputDataConfig, _Mapping]] = ..., cluster_assignment: _Optional[_Union[_cluster_assignment_pb2.ClusterAssignment, _Mapping]] = ..., interruptible: _Optional[_Union[_wrappers_pb2.BoolValue, _Mapping]] = ..., overwrite_cache: bool = ..., envs: _Optional[_Union[_common_pb2.Envs, _Mapping]] = ..., tags: _Optional[_Iterable[str]] = ..., execution_env_assignments: _Optional[_Iterable[_Union[_execution_envs_pb2.ExecutionEnvAssignment, _Mapping]]] = ...) -> None: ... class ExecutionTerminateRequest(_message.Message): __slots__ = ["id", "cause"] diff --git a/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.py b/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.py index 68d9cdbd65..7f830ca25e 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.py @@ -12,6 +12,7 @@ from flyteidl.core import execution_pb2 as flyteidl_dot_core_dot_execution__pb2 +from flyteidl.core import execution_envs_pb2 as flyteidl_dot_core_dot_execution__envs__pb2 from flyteidl.core import literals_pb2 as flyteidl_dot_core_dot_literals__pb2 from flyteidl.core import identifier_pb2 as flyteidl_dot_core_dot_identifier__pb2 from flyteidl.core import interface_pb2 as flyteidl_dot_core_dot_interface__pb2 @@ -23,7 +24,7 @@ from google.protobuf import wrappers_pb2 as google_dot_protobuf_dot_wrappers__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n flyteidl/admin/launch_plan.proto\x12\x0e\x66lyteidl.admin\x1a\x1d\x66lyteidl/core/execution.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1d\x66lyteidl/core/interface.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1d\x66lyteidl/admin/schedule.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\"x\n\x17LaunchPlanCreateRequest\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12\x32\n\x04spec\x18\x02 \x01(\x0b\x32\x1e.flyteidl.admin.LaunchPlanSpecR\x04spec\"\x1a\n\x18LaunchPlanCreateResponse\"\xa8\x01\n\nLaunchPlan\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12\x32\n\x04spec\x18\x02 \x01(\x0b\x32\x1e.flyteidl.admin.LaunchPlanSpecR\x04spec\x12;\n\x07\x63losure\x18\x03 \x01(\x0b\x32!.flyteidl.admin.LaunchPlanClosureR\x07\x63losure\"e\n\x0eLaunchPlanList\x12=\n\x0claunch_plans\x18\x01 \x03(\x0b\x32\x1a.flyteidl.admin.LaunchPlanR\x0blaunchPlans\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"v\n\x04\x41uth\x12,\n\x12\x61ssumable_iam_role\x18\x01 \x01(\tR\x10\x61ssumableIamRole\x12<\n\x1akubernetes_service_account\x18\x02 \x01(\tR\x18kubernetesServiceAccount:\x02\x18\x01\"\xbd\x07\n\x0eLaunchPlanSpec\x12:\n\x0bworkflow_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nworkflowId\x12K\n\x0f\x65ntity_metadata\x18\x02 \x01(\x0b\x32\".flyteidl.admin.LaunchPlanMetadataR\x0e\x65ntityMetadata\x12\x42\n\x0e\x64\x65\x66\x61ult_inputs\x18\x03 \x01(\x0b\x32\x1b.flyteidl.core.ParameterMapR\rdefaultInputs\x12<\n\x0c\x66ixed_inputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ixedInputs\x12\x16\n\x04role\x18\x05 \x01(\tB\x02\x18\x01R\x04role\x12.\n\x06labels\x18\x06 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x07 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12,\n\x04\x61uth\x18\x08 \x01(\x0b\x32\x14.flyteidl.admin.AuthB\x02\x18\x01R\x04\x61uth\x12\x39\n\tauth_role\x18\t \x01(\x0b\x32\x18.flyteidl.admin.AuthRoleB\x02\x18\x01R\x08\x61uthRole\x12I\n\x10security_context\x18\n \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12M\n\x12quality_of_service\x18\x10 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceR\x10qualityOfService\x12X\n\x16raw_output_data_config\x18\x11 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12\'\n\x0fmax_parallelism\x18\x12 \x01(\x05R\x0emaxParallelism\x12@\n\rinterruptible\x18\x13 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x14 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x15 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\"\xcd\x02\n\x11LaunchPlanClosure\x12\x35\n\x05state\x18\x01 \x01(\x0e\x32\x1f.flyteidl.admin.LaunchPlanStateR\x05state\x12\x44\n\x0f\x65xpected_inputs\x18\x02 \x01(\x0b\x32\x1b.flyteidl.core.ParameterMapR\x0e\x65xpectedInputs\x12\x45\n\x10\x65xpected_outputs\x18\x03 \x01(\x0b\x32\x1a.flyteidl.core.VariableMapR\x0f\x65xpectedOutputs\x12\x39\n\ncreated_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\"\xd1\x01\n\x12LaunchPlanMetadata\x12\x34\n\x08schedule\x18\x01 \x01(\x0b\x32\x18.flyteidl.admin.ScheduleR\x08schedule\x12\x42\n\rnotifications\x18\x02 \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\x12\x41\n\x11launch_conditions\x18\x03 \x01(\x0b\x32\x14.google.protobuf.AnyR\x10launchConditions\"{\n\x17LaunchPlanUpdateRequest\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12\x35\n\x05state\x18\x02 \x01(\x0e\x32\x1f.flyteidl.admin.LaunchPlanStateR\x05state\"\x1a\n\x18LaunchPlanUpdateResponse\"P\n\x17\x41\x63tiveLaunchPlanRequest\x12\x35\n\x02id\x18\x01 \x01(\x0b\x32%.flyteidl.admin.NamedEntityIdentifierR\x02id\"\xbc\x01\n\x1b\x41\x63tiveLaunchPlanListRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x14\n\x05limit\x18\x03 \x01(\rR\x05limit\x12\x14\n\x05token\x18\x04 \x01(\tR\x05token\x12-\n\x07sort_by\x18\x05 \x01(\x0b\x32\x14.flyteidl.admin.SortR\x06sortBy\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org*+\n\x0fLaunchPlanState\x12\x0c\n\x08INACTIVE\x10\x00\x12\n\n\x06\x41\x43TIVE\x10\x01\x42\xbb\x01\n\x12\x63om.flyteidl.adminB\x0fLaunchPlanProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n flyteidl/admin/launch_plan.proto\x12\x0e\x66lyteidl.admin\x1a\x1d\x66lyteidl/core/execution.proto\x1a\"flyteidl/core/execution_envs.proto\x1a\x1c\x66lyteidl/core/literals.proto\x1a\x1e\x66lyteidl/core/identifier.proto\x1a\x1d\x66lyteidl/core/interface.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1d\x66lyteidl/admin/schedule.proto\x1a\x1b\x66lyteidl/admin/common.proto\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\"x\n\x17LaunchPlanCreateRequest\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12\x32\n\x04spec\x18\x02 \x01(\x0b\x32\x1e.flyteidl.admin.LaunchPlanSpecR\x04spec\"\x1a\n\x18LaunchPlanCreateResponse\"\xa8\x01\n\nLaunchPlan\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12\x32\n\x04spec\x18\x02 \x01(\x0b\x32\x1e.flyteidl.admin.LaunchPlanSpecR\x04spec\x12;\n\x07\x63losure\x18\x03 \x01(\x0b\x32!.flyteidl.admin.LaunchPlanClosureR\x07\x63losure\"e\n\x0eLaunchPlanList\x12=\n\x0claunch_plans\x18\x01 \x03(\x0b\x32\x1a.flyteidl.admin.LaunchPlanR\x0blaunchPlans\x12\x14\n\x05token\x18\x02 \x01(\tR\x05token\"v\n\x04\x41uth\x12,\n\x12\x61ssumable_iam_role\x18\x01 \x01(\tR\x10\x61ssumableIamRole\x12<\n\x1akubernetes_service_account\x18\x02 \x01(\tR\x18kubernetesServiceAccount:\x02\x18\x01\"\xa0\x08\n\x0eLaunchPlanSpec\x12:\n\x0bworkflow_id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\nworkflowId\x12K\n\x0f\x65ntity_metadata\x18\x02 \x01(\x0b\x32\".flyteidl.admin.LaunchPlanMetadataR\x0e\x65ntityMetadata\x12\x42\n\x0e\x64\x65\x66\x61ult_inputs\x18\x03 \x01(\x0b\x32\x1b.flyteidl.core.ParameterMapR\rdefaultInputs\x12<\n\x0c\x66ixed_inputs\x18\x04 \x01(\x0b\x32\x19.flyteidl.core.LiteralMapR\x0b\x66ixedInputs\x12\x16\n\x04role\x18\x05 \x01(\tB\x02\x18\x01R\x04role\x12.\n\x06labels\x18\x06 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x07 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12,\n\x04\x61uth\x18\x08 \x01(\x0b\x32\x14.flyteidl.admin.AuthB\x02\x18\x01R\x04\x61uth\x12\x39\n\tauth_role\x18\t \x01(\x0b\x32\x18.flyteidl.admin.AuthRoleB\x02\x18\x01R\x08\x61uthRole\x12I\n\x10security_context\x18\n \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12M\n\x12quality_of_service\x18\x10 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceR\x10qualityOfService\x12X\n\x16raw_output_data_config\x18\x11 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12\'\n\x0fmax_parallelism\x18\x12 \x01(\x05R\x0emaxParallelism\x12@\n\rinterruptible\x18\x13 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x14 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x15 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\x12\x61\n\x19\x65xecution_env_assignments\x18\x16 \x03(\x0b\x32%.flyteidl.core.ExecutionEnvAssignmentR\x17\x65xecutionEnvAssignments\"\xcd\x02\n\x11LaunchPlanClosure\x12\x35\n\x05state\x18\x01 \x01(\x0e\x32\x1f.flyteidl.admin.LaunchPlanStateR\x05state\x12\x44\n\x0f\x65xpected_inputs\x18\x02 \x01(\x0b\x32\x1b.flyteidl.core.ParameterMapR\x0e\x65xpectedInputs\x12\x45\n\x10\x65xpected_outputs\x18\x03 \x01(\x0b\x32\x1a.flyteidl.core.VariableMapR\x0f\x65xpectedOutputs\x12\x39\n\ncreated_at\x18\x04 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tcreatedAt\x12\x39\n\nupdated_at\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\tupdatedAt\"\xd1\x01\n\x12LaunchPlanMetadata\x12\x34\n\x08schedule\x18\x01 \x01(\x0b\x32\x18.flyteidl.admin.ScheduleR\x08schedule\x12\x42\n\rnotifications\x18\x02 \x03(\x0b\x32\x1c.flyteidl.admin.NotificationR\rnotifications\x12\x41\n\x11launch_conditions\x18\x03 \x01(\x0b\x32\x14.google.protobuf.AnyR\x10launchConditions\"{\n\x17LaunchPlanUpdateRequest\x12)\n\x02id\x18\x01 \x01(\x0b\x32\x19.flyteidl.core.IdentifierR\x02id\x12\x35\n\x05state\x18\x02 \x01(\x0e\x32\x1f.flyteidl.admin.LaunchPlanStateR\x05state\"\x1a\n\x18LaunchPlanUpdateResponse\"P\n\x17\x41\x63tiveLaunchPlanRequest\x12\x35\n\x02id\x18\x01 \x01(\x0b\x32%.flyteidl.admin.NamedEntityIdentifierR\x02id\"\xbc\x01\n\x1b\x41\x63tiveLaunchPlanListRequest\x12\x18\n\x07project\x18\x01 \x01(\tR\x07project\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x14\n\x05limit\x18\x03 \x01(\rR\x05limit\x12\x14\n\x05token\x18\x04 \x01(\tR\x05token\x12-\n\x07sort_by\x18\x05 \x01(\x0b\x32\x14.flyteidl.admin.SortR\x06sortBy\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org*+\n\x0fLaunchPlanState\x12\x0c\n\x08INACTIVE\x10\x00\x12\n\n\x06\x41\x43TIVE\x10\x01\x42\xbb\x01\n\x12\x63om.flyteidl.adminB\x0fLaunchPlanProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -40,30 +41,30 @@ _LAUNCHPLANSPEC.fields_by_name['auth']._serialized_options = b'\030\001' _LAUNCHPLANSPEC.fields_by_name['auth_role']._options = None _LAUNCHPLANSPEC.fields_by_name['auth_role']._serialized_options = b'\030\001' - _globals['_LAUNCHPLANSTATE']._serialized_start=2836 - _globals['_LAUNCHPLANSTATE']._serialized_end=2879 - _globals['_LAUNCHPLANCREATEREQUEST']._serialized_start=358 - _globals['_LAUNCHPLANCREATEREQUEST']._serialized_end=478 - _globals['_LAUNCHPLANCREATERESPONSE']._serialized_start=480 - _globals['_LAUNCHPLANCREATERESPONSE']._serialized_end=506 - _globals['_LAUNCHPLAN']._serialized_start=509 - _globals['_LAUNCHPLAN']._serialized_end=677 - _globals['_LAUNCHPLANLIST']._serialized_start=679 - _globals['_LAUNCHPLANLIST']._serialized_end=780 - _globals['_AUTH']._serialized_start=782 - _globals['_AUTH']._serialized_end=900 - _globals['_LAUNCHPLANSPEC']._serialized_start=903 - _globals['_LAUNCHPLANSPEC']._serialized_end=1860 - _globals['_LAUNCHPLANCLOSURE']._serialized_start=1863 - _globals['_LAUNCHPLANCLOSURE']._serialized_end=2196 - _globals['_LAUNCHPLANMETADATA']._serialized_start=2199 - _globals['_LAUNCHPLANMETADATA']._serialized_end=2408 - _globals['_LAUNCHPLANUPDATEREQUEST']._serialized_start=2410 - _globals['_LAUNCHPLANUPDATEREQUEST']._serialized_end=2533 - _globals['_LAUNCHPLANUPDATERESPONSE']._serialized_start=2535 - _globals['_LAUNCHPLANUPDATERESPONSE']._serialized_end=2561 - _globals['_ACTIVELAUNCHPLANREQUEST']._serialized_start=2563 - _globals['_ACTIVELAUNCHPLANREQUEST']._serialized_end=2643 - _globals['_ACTIVELAUNCHPLANLISTREQUEST']._serialized_start=2646 - _globals['_ACTIVELAUNCHPLANLISTREQUEST']._serialized_end=2834 + _globals['_LAUNCHPLANSTATE']._serialized_start=2971 + _globals['_LAUNCHPLANSTATE']._serialized_end=3014 + _globals['_LAUNCHPLANCREATEREQUEST']._serialized_start=394 + _globals['_LAUNCHPLANCREATEREQUEST']._serialized_end=514 + _globals['_LAUNCHPLANCREATERESPONSE']._serialized_start=516 + _globals['_LAUNCHPLANCREATERESPONSE']._serialized_end=542 + _globals['_LAUNCHPLAN']._serialized_start=545 + _globals['_LAUNCHPLAN']._serialized_end=713 + _globals['_LAUNCHPLANLIST']._serialized_start=715 + _globals['_LAUNCHPLANLIST']._serialized_end=816 + _globals['_AUTH']._serialized_start=818 + _globals['_AUTH']._serialized_end=936 + _globals['_LAUNCHPLANSPEC']._serialized_start=939 + _globals['_LAUNCHPLANSPEC']._serialized_end=1995 + _globals['_LAUNCHPLANCLOSURE']._serialized_start=1998 + _globals['_LAUNCHPLANCLOSURE']._serialized_end=2331 + _globals['_LAUNCHPLANMETADATA']._serialized_start=2334 + _globals['_LAUNCHPLANMETADATA']._serialized_end=2543 + _globals['_LAUNCHPLANUPDATEREQUEST']._serialized_start=2545 + _globals['_LAUNCHPLANUPDATEREQUEST']._serialized_end=2668 + _globals['_LAUNCHPLANUPDATERESPONSE']._serialized_start=2670 + _globals['_LAUNCHPLANUPDATERESPONSE']._serialized_end=2696 + _globals['_ACTIVELAUNCHPLANREQUEST']._serialized_start=2698 + _globals['_ACTIVELAUNCHPLANREQUEST']._serialized_end=2778 + _globals['_ACTIVELAUNCHPLANLISTREQUEST']._serialized_start=2781 + _globals['_ACTIVELAUNCHPLANLISTREQUEST']._serialized_end=2969 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.pyi index a047c8d473..46e7669559 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/admin/launch_plan_pb2.pyi @@ -1,4 +1,5 @@ from flyteidl.core import execution_pb2 as _execution_pb2 +from flyteidl.core import execution_envs_pb2 as _execution_envs_pb2 from flyteidl.core import literals_pb2 as _literals_pb2 from flyteidl.core import identifier_pb2 as _identifier_pb2 from flyteidl.core import interface_pb2 as _interface_pb2 @@ -62,7 +63,7 @@ class Auth(_message.Message): def __init__(self, assumable_iam_role: _Optional[str] = ..., kubernetes_service_account: _Optional[str] = ...) -> None: ... class LaunchPlanSpec(_message.Message): - __slots__ = ["workflow_id", "entity_metadata", "default_inputs", "fixed_inputs", "role", "labels", "annotations", "auth", "auth_role", "security_context", "quality_of_service", "raw_output_data_config", "max_parallelism", "interruptible", "overwrite_cache", "envs"] + __slots__ = ["workflow_id", "entity_metadata", "default_inputs", "fixed_inputs", "role", "labels", "annotations", "auth", "auth_role", "security_context", "quality_of_service", "raw_output_data_config", "max_parallelism", "interruptible", "overwrite_cache", "envs", "execution_env_assignments"] WORKFLOW_ID_FIELD_NUMBER: _ClassVar[int] ENTITY_METADATA_FIELD_NUMBER: _ClassVar[int] DEFAULT_INPUTS_FIELD_NUMBER: _ClassVar[int] @@ -79,6 +80,7 @@ class LaunchPlanSpec(_message.Message): INTERRUPTIBLE_FIELD_NUMBER: _ClassVar[int] OVERWRITE_CACHE_FIELD_NUMBER: _ClassVar[int] ENVS_FIELD_NUMBER: _ClassVar[int] + EXECUTION_ENV_ASSIGNMENTS_FIELD_NUMBER: _ClassVar[int] workflow_id: _identifier_pb2.Identifier entity_metadata: LaunchPlanMetadata default_inputs: _interface_pb2.ParameterMap @@ -95,7 +97,8 @@ class LaunchPlanSpec(_message.Message): interruptible: _wrappers_pb2.BoolValue overwrite_cache: bool envs: _common_pb2.Envs - def __init__(self, workflow_id: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., entity_metadata: _Optional[_Union[LaunchPlanMetadata, _Mapping]] = ..., default_inputs: _Optional[_Union[_interface_pb2.ParameterMap, _Mapping]] = ..., fixed_inputs: _Optional[_Union[_literals_pb2.LiteralMap, _Mapping]] = ..., role: _Optional[str] = ..., labels: _Optional[_Union[_common_pb2.Labels, _Mapping]] = ..., annotations: _Optional[_Union[_common_pb2.Annotations, _Mapping]] = ..., auth: _Optional[_Union[Auth, _Mapping]] = ..., auth_role: _Optional[_Union[_common_pb2.AuthRole, _Mapping]] = ..., security_context: _Optional[_Union[_security_pb2.SecurityContext, _Mapping]] = ..., quality_of_service: _Optional[_Union[_execution_pb2.QualityOfService, _Mapping]] = ..., raw_output_data_config: _Optional[_Union[_common_pb2.RawOutputDataConfig, _Mapping]] = ..., max_parallelism: _Optional[int] = ..., interruptible: _Optional[_Union[_wrappers_pb2.BoolValue, _Mapping]] = ..., overwrite_cache: bool = ..., envs: _Optional[_Union[_common_pb2.Envs, _Mapping]] = ...) -> None: ... + execution_env_assignments: _containers.RepeatedCompositeFieldContainer[_execution_envs_pb2.ExecutionEnvAssignment] + def __init__(self, workflow_id: _Optional[_Union[_identifier_pb2.Identifier, _Mapping]] = ..., entity_metadata: _Optional[_Union[LaunchPlanMetadata, _Mapping]] = ..., default_inputs: _Optional[_Union[_interface_pb2.ParameterMap, _Mapping]] = ..., fixed_inputs: _Optional[_Union[_literals_pb2.LiteralMap, _Mapping]] = ..., role: _Optional[str] = ..., labels: _Optional[_Union[_common_pb2.Labels, _Mapping]] = ..., annotations: _Optional[_Union[_common_pb2.Annotations, _Mapping]] = ..., auth: _Optional[_Union[Auth, _Mapping]] = ..., auth_role: _Optional[_Union[_common_pb2.AuthRole, _Mapping]] = ..., security_context: _Optional[_Union[_security_pb2.SecurityContext, _Mapping]] = ..., quality_of_service: _Optional[_Union[_execution_pb2.QualityOfService, _Mapping]] = ..., raw_output_data_config: _Optional[_Union[_common_pb2.RawOutputDataConfig, _Mapping]] = ..., max_parallelism: _Optional[int] = ..., interruptible: _Optional[_Union[_wrappers_pb2.BoolValue, _Mapping]] = ..., overwrite_cache: bool = ..., envs: _Optional[_Union[_common_pb2.Envs, _Mapping]] = ..., execution_env_assignments: _Optional[_Iterable[_Union[_execution_envs_pb2.ExecutionEnvAssignment, _Mapping]]] = ...) -> None: ... class LaunchPlanClosure(_message.Message): __slots__ = ["state", "expected_inputs", "expected_outputs", "created_at", "updated_at"] diff --git a/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.py b/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.py index 94748a4932..012e208c06 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.py +++ b/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.py @@ -14,11 +14,12 @@ from flyteidl.admin import common_pb2 as flyteidl_dot_admin_dot_common__pb2 from flyteidl.admin import cluster_assignment_pb2 as flyteidl_dot_admin_dot_cluster__assignment__pb2 from flyteidl.core import execution_pb2 as flyteidl_dot_core_dot_execution__pb2 +from flyteidl.core import execution_envs_pb2 as flyteidl_dot_core_dot_execution__envs__pb2 from flyteidl.core import security_pb2 as flyteidl_dot_core_dot_security__pb2 from google.protobuf import wrappers_pb2 as google_dot_protobuf_dot_wrappers__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\'flyteidl/admin/matchable_resource.proto\x12\x0e\x66lyteidl.admin\x1a\x1b\x66lyteidl/admin/common.proto\x1a\'flyteidl/admin/cluster_assignment.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1egoogle/protobuf/wrappers.proto\"\x95\x01\n\x10TaskResourceSpec\x12\x10\n\x03\x63pu\x18\x01 \x01(\tR\x03\x63pu\x12\x10\n\x03gpu\x18\x02 \x01(\tR\x03gpu\x12\x16\n\x06memory\x18\x03 \x01(\tR\x06memory\x12\x18\n\x07storage\x18\x04 \x01(\tR\x07storage\x12+\n\x11\x65phemeral_storage\x18\x05 \x01(\tR\x10\x65phemeralStorage\"\x90\x01\n\x16TaskResourceAttributes\x12<\n\x08\x64\x65\x66\x61ults\x18\x01 \x01(\x0b\x32 .flyteidl.admin.TaskResourceSpecR\x08\x64\x65\x66\x61ults\x12\x38\n\x06limits\x18\x02 \x01(\x0b\x32 .flyteidl.admin.TaskResourceSpecR\x06limits\"\xb5\x01\n\x19\x43lusterResourceAttributes\x12Y\n\nattributes\x18\x01 \x03(\x0b\x32\x39.flyteidl.admin.ClusterResourceAttributes.AttributesEntryR\nattributes\x1a=\n\x0f\x41ttributesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\".\n\x18\x45xecutionQueueAttributes\x12\x12\n\x04tags\x18\x01 \x03(\tR\x04tags\"-\n\x15\x45xecutionClusterLabel\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\"\xec\x01\n\x0ePluginOverride\x12\x1b\n\ttask_type\x18\x01 \x01(\tR\x08taskType\x12\x1b\n\tplugin_id\x18\x02 \x03(\tR\x08pluginId\x12l\n\x17missing_plugin_behavior\x18\x04 \x01(\x0e\x32\x34.flyteidl.admin.PluginOverride.MissingPluginBehaviorR\x15missingPluginBehavior\"2\n\x15MissingPluginBehavior\x12\x08\n\x04\x46\x41IL\x10\x00\x12\x0f\n\x0bUSE_DEFAULT\x10\x01\"O\n\x0fPluginOverrides\x12<\n\toverrides\x18\x01 \x03(\x0b\x32\x1e.flyteidl.admin.PluginOverrideR\toverrides\"\xeb\x03\n\x17WorkflowExecutionConfig\x12\'\n\x0fmax_parallelism\x18\x01 \x01(\x05R\x0emaxParallelism\x12I\n\x10security_context\x18\x02 \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12X\n\x16raw_output_data_config\x18\x03 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12.\n\x06labels\x18\x04 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x05 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12@\n\rinterruptible\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x07 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x08 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\"\x94\x06\n\x12MatchingAttributes\x12\x62\n\x18task_resource_attributes\x18\x01 \x01(\x0b\x32&.flyteidl.admin.TaskResourceAttributesH\x00R\x16taskResourceAttributes\x12k\n\x1b\x63luster_resource_attributes\x18\x02 \x01(\x0b\x32).flyteidl.admin.ClusterResourceAttributesH\x00R\x19\x63lusterResourceAttributes\x12h\n\x1a\x65xecution_queue_attributes\x18\x03 \x01(\x0b\x32(.flyteidl.admin.ExecutionQueueAttributesH\x00R\x18\x65xecutionQueueAttributes\x12_\n\x17\x65xecution_cluster_label\x18\x04 \x01(\x0b\x32%.flyteidl.admin.ExecutionClusterLabelH\x00R\x15\x65xecutionClusterLabel\x12O\n\x12quality_of_service\x18\x05 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceH\x00R\x10qualityOfService\x12L\n\x10plugin_overrides\x18\x06 \x01(\x0b\x32\x1f.flyteidl.admin.PluginOverridesH\x00R\x0fpluginOverrides\x12\x65\n\x19workflow_execution_config\x18\x07 \x01(\x0b\x32\'.flyteidl.admin.WorkflowExecutionConfigH\x00R\x17workflowExecutionConfig\x12R\n\x12\x63luster_assignment\x18\x08 \x01(\x0b\x32!.flyteidl.admin.ClusterAssignmentH\x00R\x11\x63lusterAssignmentB\x08\n\x06target\"\xe7\x01\n MatchableAttributesConfiguration\x12\x42\n\nattributes\x18\x01 \x01(\x0b\x32\".flyteidl.admin.MatchingAttributesR\nattributes\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x18\n\x07project\x18\x03 \x01(\tR\x07project\x12\x1a\n\x08workflow\x18\x04 \x01(\tR\x08workflow\x12\x1f\n\x0blaunch_plan\x18\x05 \x01(\tR\nlaunchPlan\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org\"z\n\x1eListMatchableAttributesRequest\x12\x46\n\rresource_type\x18\x01 \x01(\x0e\x32!.flyteidl.admin.MatchableResourceR\x0cresourceType\x12\x10\n\x03org\x18\x02 \x01(\tR\x03org\"{\n\x1fListMatchableAttributesResponse\x12X\n\x0e\x63onfigurations\x18\x01 \x03(\x0b\x32\x30.flyteidl.admin.MatchableAttributesConfigurationR\x0e\x63onfigurations*\xe0\x01\n\x11MatchableResource\x12\x11\n\rTASK_RESOURCE\x10\x00\x12\x14\n\x10\x43LUSTER_RESOURCE\x10\x01\x12\x13\n\x0f\x45XECUTION_QUEUE\x10\x02\x12\x1b\n\x17\x45XECUTION_CLUSTER_LABEL\x10\x03\x12$\n QUALITY_OF_SERVICE_SPECIFICATION\x10\x04\x12\x13\n\x0fPLUGIN_OVERRIDE\x10\x05\x12\x1d\n\x19WORKFLOW_EXECUTION_CONFIG\x10\x06\x12\x16\n\x12\x43LUSTER_ASSIGNMENT\x10\x07\x42\xc2\x01\n\x12\x63om.flyteidl.adminB\x16MatchableResourceProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\'flyteidl/admin/matchable_resource.proto\x12\x0e\x66lyteidl.admin\x1a\x1b\x66lyteidl/admin/common.proto\x1a\'flyteidl/admin/cluster_assignment.proto\x1a\x1d\x66lyteidl/core/execution.proto\x1a\"flyteidl/core/execution_envs.proto\x1a\x1c\x66lyteidl/core/security.proto\x1a\x1egoogle/protobuf/wrappers.proto\"\x95\x01\n\x10TaskResourceSpec\x12\x10\n\x03\x63pu\x18\x01 \x01(\tR\x03\x63pu\x12\x10\n\x03gpu\x18\x02 \x01(\tR\x03gpu\x12\x16\n\x06memory\x18\x03 \x01(\tR\x06memory\x12\x18\n\x07storage\x18\x04 \x01(\tR\x07storage\x12+\n\x11\x65phemeral_storage\x18\x05 \x01(\tR\x10\x65phemeralStorage\"\x90\x01\n\x16TaskResourceAttributes\x12<\n\x08\x64\x65\x66\x61ults\x18\x01 \x01(\x0b\x32 .flyteidl.admin.TaskResourceSpecR\x08\x64\x65\x66\x61ults\x12\x38\n\x06limits\x18\x02 \x01(\x0b\x32 .flyteidl.admin.TaskResourceSpecR\x06limits\"\xb5\x01\n\x19\x43lusterResourceAttributes\x12Y\n\nattributes\x18\x01 \x03(\x0b\x32\x39.flyteidl.admin.ClusterResourceAttributes.AttributesEntryR\nattributes\x1a=\n\x0f\x41ttributesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\".\n\x18\x45xecutionQueueAttributes\x12\x12\n\x04tags\x18\x01 \x03(\tR\x04tags\"-\n\x15\x45xecutionClusterLabel\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\"\xec\x01\n\x0ePluginOverride\x12\x1b\n\ttask_type\x18\x01 \x01(\tR\x08taskType\x12\x1b\n\tplugin_id\x18\x02 \x03(\tR\x08pluginId\x12l\n\x17missing_plugin_behavior\x18\x04 \x01(\x0e\x32\x34.flyteidl.admin.PluginOverride.MissingPluginBehaviorR\x15missingPluginBehavior\"2\n\x15MissingPluginBehavior\x12\x08\n\x04\x46\x41IL\x10\x00\x12\x0f\n\x0bUSE_DEFAULT\x10\x01\"O\n\x0fPluginOverrides\x12<\n\toverrides\x18\x01 \x03(\x0b\x32\x1e.flyteidl.admin.PluginOverrideR\toverrides\"\xce\x04\n\x17WorkflowExecutionConfig\x12\'\n\x0fmax_parallelism\x18\x01 \x01(\x05R\x0emaxParallelism\x12I\n\x10security_context\x18\x02 \x01(\x0b\x32\x1e.flyteidl.core.SecurityContextR\x0fsecurityContext\x12X\n\x16raw_output_data_config\x18\x03 \x01(\x0b\x32#.flyteidl.admin.RawOutputDataConfigR\x13rawOutputDataConfig\x12.\n\x06labels\x18\x04 \x01(\x0b\x32\x16.flyteidl.admin.LabelsR\x06labels\x12=\n\x0b\x61nnotations\x18\x05 \x01(\x0b\x32\x1b.flyteidl.admin.AnnotationsR\x0b\x61nnotations\x12@\n\rinterruptible\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.BoolValueR\rinterruptible\x12\'\n\x0foverwrite_cache\x18\x07 \x01(\x08R\x0eoverwriteCache\x12(\n\x04\x65nvs\x18\x08 \x01(\x0b\x32\x14.flyteidl.admin.EnvsR\x04\x65nvs\x12\x61\n\x19\x65xecution_env_assignments\x18\t \x03(\x0b\x32%.flyteidl.core.ExecutionEnvAssignmentR\x17\x65xecutionEnvAssignments\"\x94\x06\n\x12MatchingAttributes\x12\x62\n\x18task_resource_attributes\x18\x01 \x01(\x0b\x32&.flyteidl.admin.TaskResourceAttributesH\x00R\x16taskResourceAttributes\x12k\n\x1b\x63luster_resource_attributes\x18\x02 \x01(\x0b\x32).flyteidl.admin.ClusterResourceAttributesH\x00R\x19\x63lusterResourceAttributes\x12h\n\x1a\x65xecution_queue_attributes\x18\x03 \x01(\x0b\x32(.flyteidl.admin.ExecutionQueueAttributesH\x00R\x18\x65xecutionQueueAttributes\x12_\n\x17\x65xecution_cluster_label\x18\x04 \x01(\x0b\x32%.flyteidl.admin.ExecutionClusterLabelH\x00R\x15\x65xecutionClusterLabel\x12O\n\x12quality_of_service\x18\x05 \x01(\x0b\x32\x1f.flyteidl.core.QualityOfServiceH\x00R\x10qualityOfService\x12L\n\x10plugin_overrides\x18\x06 \x01(\x0b\x32\x1f.flyteidl.admin.PluginOverridesH\x00R\x0fpluginOverrides\x12\x65\n\x19workflow_execution_config\x18\x07 \x01(\x0b\x32\'.flyteidl.admin.WorkflowExecutionConfigH\x00R\x17workflowExecutionConfig\x12R\n\x12\x63luster_assignment\x18\x08 \x01(\x0b\x32!.flyteidl.admin.ClusterAssignmentH\x00R\x11\x63lusterAssignmentB\x08\n\x06target\"\xe7\x01\n MatchableAttributesConfiguration\x12\x42\n\nattributes\x18\x01 \x01(\x0b\x32\".flyteidl.admin.MatchingAttributesR\nattributes\x12\x16\n\x06\x64omain\x18\x02 \x01(\tR\x06\x64omain\x12\x18\n\x07project\x18\x03 \x01(\tR\x07project\x12\x1a\n\x08workflow\x18\x04 \x01(\tR\x08workflow\x12\x1f\n\x0blaunch_plan\x18\x05 \x01(\tR\nlaunchPlan\x12\x10\n\x03org\x18\x06 \x01(\tR\x03org\"z\n\x1eListMatchableAttributesRequest\x12\x46\n\rresource_type\x18\x01 \x01(\x0e\x32!.flyteidl.admin.MatchableResourceR\x0cresourceType\x12\x10\n\x03org\x18\x02 \x01(\tR\x03org\"{\n\x1fListMatchableAttributesResponse\x12X\n\x0e\x63onfigurations\x18\x01 \x03(\x0b\x32\x30.flyteidl.admin.MatchableAttributesConfigurationR\x0e\x63onfigurations*\xe0\x01\n\x11MatchableResource\x12\x11\n\rTASK_RESOURCE\x10\x00\x12\x14\n\x10\x43LUSTER_RESOURCE\x10\x01\x12\x13\n\x0f\x45XECUTION_QUEUE\x10\x02\x12\x1b\n\x17\x45XECUTION_CLUSTER_LABEL\x10\x03\x12$\n QUALITY_OF_SERVICE_SPECIFICATION\x10\x04\x12\x13\n\x0fPLUGIN_OVERRIDE\x10\x05\x12\x1d\n\x19WORKFLOW_EXECUTION_CONFIG\x10\x06\x12\x16\n\x12\x43LUSTER_ASSIGNMENT\x10\x07\x42\xc2\x01\n\x12\x63om.flyteidl.adminB\x16MatchableResourceProtoP\x01Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\xa2\x02\x03\x46\x41X\xaa\x02\x0e\x46lyteidl.Admin\xca\x02\x0e\x46lyteidl\\Admin\xe2\x02\x1a\x46lyteidl\\Admin\\GPBMetadata\xea\x02\x0f\x46lyteidl::Adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -29,34 +30,34 @@ DESCRIPTOR._serialized_options = b'\n\022com.flyteidl.adminB\026MatchableResourceProtoP\001Z;github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin\242\002\003FAX\252\002\016Flyteidl.Admin\312\002\016Flyteidl\\Admin\342\002\032Flyteidl\\Admin\\GPBMetadata\352\002\017Flyteidl::Admin' _CLUSTERRESOURCEATTRIBUTES_ATTRIBUTESENTRY._options = None _CLUSTERRESOURCEATTRIBUTES_ATTRIBUTESENTRY._serialized_options = b'8\001' - _globals['_MATCHABLERESOURCE']._serialized_start=2889 - _globals['_MATCHABLERESOURCE']._serialized_end=3113 - _globals['_TASKRESOURCESPEC']._serialized_start=223 - _globals['_TASKRESOURCESPEC']._serialized_end=372 - _globals['_TASKRESOURCEATTRIBUTES']._serialized_start=375 - _globals['_TASKRESOURCEATTRIBUTES']._serialized_end=519 - _globals['_CLUSTERRESOURCEATTRIBUTES']._serialized_start=522 - _globals['_CLUSTERRESOURCEATTRIBUTES']._serialized_end=703 - _globals['_CLUSTERRESOURCEATTRIBUTES_ATTRIBUTESENTRY']._serialized_start=642 - _globals['_CLUSTERRESOURCEATTRIBUTES_ATTRIBUTESENTRY']._serialized_end=703 - _globals['_EXECUTIONQUEUEATTRIBUTES']._serialized_start=705 - _globals['_EXECUTIONQUEUEATTRIBUTES']._serialized_end=751 - _globals['_EXECUTIONCLUSTERLABEL']._serialized_start=753 - _globals['_EXECUTIONCLUSTERLABEL']._serialized_end=798 - _globals['_PLUGINOVERRIDE']._serialized_start=801 - _globals['_PLUGINOVERRIDE']._serialized_end=1037 - _globals['_PLUGINOVERRIDE_MISSINGPLUGINBEHAVIOR']._serialized_start=987 - _globals['_PLUGINOVERRIDE_MISSINGPLUGINBEHAVIOR']._serialized_end=1037 - _globals['_PLUGINOVERRIDES']._serialized_start=1039 - _globals['_PLUGINOVERRIDES']._serialized_end=1118 - _globals['_WORKFLOWEXECUTIONCONFIG']._serialized_start=1121 - _globals['_WORKFLOWEXECUTIONCONFIG']._serialized_end=1612 - _globals['_MATCHINGATTRIBUTES']._serialized_start=1615 - _globals['_MATCHINGATTRIBUTES']._serialized_end=2403 - _globals['_MATCHABLEATTRIBUTESCONFIGURATION']._serialized_start=2406 - _globals['_MATCHABLEATTRIBUTESCONFIGURATION']._serialized_end=2637 - _globals['_LISTMATCHABLEATTRIBUTESREQUEST']._serialized_start=2639 - _globals['_LISTMATCHABLEATTRIBUTESREQUEST']._serialized_end=2761 - _globals['_LISTMATCHABLEATTRIBUTESRESPONSE']._serialized_start=2763 - _globals['_LISTMATCHABLEATTRIBUTESRESPONSE']._serialized_end=2886 + _globals['_MATCHABLERESOURCE']._serialized_start=3024 + _globals['_MATCHABLERESOURCE']._serialized_end=3248 + _globals['_TASKRESOURCESPEC']._serialized_start=259 + _globals['_TASKRESOURCESPEC']._serialized_end=408 + _globals['_TASKRESOURCEATTRIBUTES']._serialized_start=411 + _globals['_TASKRESOURCEATTRIBUTES']._serialized_end=555 + _globals['_CLUSTERRESOURCEATTRIBUTES']._serialized_start=558 + _globals['_CLUSTERRESOURCEATTRIBUTES']._serialized_end=739 + _globals['_CLUSTERRESOURCEATTRIBUTES_ATTRIBUTESENTRY']._serialized_start=678 + _globals['_CLUSTERRESOURCEATTRIBUTES_ATTRIBUTESENTRY']._serialized_end=739 + _globals['_EXECUTIONQUEUEATTRIBUTES']._serialized_start=741 + _globals['_EXECUTIONQUEUEATTRIBUTES']._serialized_end=787 + _globals['_EXECUTIONCLUSTERLABEL']._serialized_start=789 + _globals['_EXECUTIONCLUSTERLABEL']._serialized_end=834 + _globals['_PLUGINOVERRIDE']._serialized_start=837 + _globals['_PLUGINOVERRIDE']._serialized_end=1073 + _globals['_PLUGINOVERRIDE_MISSINGPLUGINBEHAVIOR']._serialized_start=1023 + _globals['_PLUGINOVERRIDE_MISSINGPLUGINBEHAVIOR']._serialized_end=1073 + _globals['_PLUGINOVERRIDES']._serialized_start=1075 + _globals['_PLUGINOVERRIDES']._serialized_end=1154 + _globals['_WORKFLOWEXECUTIONCONFIG']._serialized_start=1157 + _globals['_WORKFLOWEXECUTIONCONFIG']._serialized_end=1747 + _globals['_MATCHINGATTRIBUTES']._serialized_start=1750 + _globals['_MATCHINGATTRIBUTES']._serialized_end=2538 + _globals['_MATCHABLEATTRIBUTESCONFIGURATION']._serialized_start=2541 + _globals['_MATCHABLEATTRIBUTESCONFIGURATION']._serialized_end=2772 + _globals['_LISTMATCHABLEATTRIBUTESREQUEST']._serialized_start=2774 + _globals['_LISTMATCHABLEATTRIBUTESREQUEST']._serialized_end=2896 + _globals['_LISTMATCHABLEATTRIBUTESRESPONSE']._serialized_start=2898 + _globals['_LISTMATCHABLEATTRIBUTESRESPONSE']._serialized_end=3021 # @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.pyi index cc19f2d146..19c1d598a2 100644 --- a/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.pyi +++ b/flyteidl/gen/pb_python/flyteidl/admin/matchable_resource_pb2.pyi @@ -1,6 +1,7 @@ from flyteidl.admin import common_pb2 as _common_pb2 from flyteidl.admin import cluster_assignment_pb2 as _cluster_assignment_pb2 from flyteidl.core import execution_pb2 as _execution_pb2 +from flyteidl.core import execution_envs_pb2 as _execution_envs_pb2 from flyteidl.core import security_pb2 as _security_pb2 from google.protobuf import wrappers_pb2 as _wrappers_pb2 from google.protobuf.internal import containers as _containers @@ -100,7 +101,7 @@ class PluginOverrides(_message.Message): def __init__(self, overrides: _Optional[_Iterable[_Union[PluginOverride, _Mapping]]] = ...) -> None: ... class WorkflowExecutionConfig(_message.Message): - __slots__ = ["max_parallelism", "security_context", "raw_output_data_config", "labels", "annotations", "interruptible", "overwrite_cache", "envs"] + __slots__ = ["max_parallelism", "security_context", "raw_output_data_config", "labels", "annotations", "interruptible", "overwrite_cache", "envs", "execution_env_assignments"] MAX_PARALLELISM_FIELD_NUMBER: _ClassVar[int] SECURITY_CONTEXT_FIELD_NUMBER: _ClassVar[int] RAW_OUTPUT_DATA_CONFIG_FIELD_NUMBER: _ClassVar[int] @@ -109,6 +110,7 @@ class WorkflowExecutionConfig(_message.Message): INTERRUPTIBLE_FIELD_NUMBER: _ClassVar[int] OVERWRITE_CACHE_FIELD_NUMBER: _ClassVar[int] ENVS_FIELD_NUMBER: _ClassVar[int] + EXECUTION_ENV_ASSIGNMENTS_FIELD_NUMBER: _ClassVar[int] max_parallelism: int security_context: _security_pb2.SecurityContext raw_output_data_config: _common_pb2.RawOutputDataConfig @@ -117,7 +119,8 @@ class WorkflowExecutionConfig(_message.Message): interruptible: _wrappers_pb2.BoolValue overwrite_cache: bool envs: _common_pb2.Envs - def __init__(self, max_parallelism: _Optional[int] = ..., security_context: _Optional[_Union[_security_pb2.SecurityContext, _Mapping]] = ..., raw_output_data_config: _Optional[_Union[_common_pb2.RawOutputDataConfig, _Mapping]] = ..., labels: _Optional[_Union[_common_pb2.Labels, _Mapping]] = ..., annotations: _Optional[_Union[_common_pb2.Annotations, _Mapping]] = ..., interruptible: _Optional[_Union[_wrappers_pb2.BoolValue, _Mapping]] = ..., overwrite_cache: bool = ..., envs: _Optional[_Union[_common_pb2.Envs, _Mapping]] = ...) -> None: ... + execution_env_assignments: _containers.RepeatedCompositeFieldContainer[_execution_envs_pb2.ExecutionEnvAssignment] + def __init__(self, max_parallelism: _Optional[int] = ..., security_context: _Optional[_Union[_security_pb2.SecurityContext, _Mapping]] = ..., raw_output_data_config: _Optional[_Union[_common_pb2.RawOutputDataConfig, _Mapping]] = ..., labels: _Optional[_Union[_common_pb2.Labels, _Mapping]] = ..., annotations: _Optional[_Union[_common_pb2.Annotations, _Mapping]] = ..., interruptible: _Optional[_Union[_wrappers_pb2.BoolValue, _Mapping]] = ..., overwrite_cache: bool = ..., envs: _Optional[_Union[_common_pb2.Envs, _Mapping]] = ..., execution_env_assignments: _Optional[_Iterable[_Union[_execution_envs_pb2.ExecutionEnvAssignment, _Mapping]]] = ...) -> None: ... class MatchingAttributes(_message.Message): __slots__ = ["task_resource_attributes", "cluster_resource_attributes", "execution_queue_attributes", "execution_cluster_label", "quality_of_service", "plugin_overrides", "workflow_execution_config", "cluster_assignment"] diff --git a/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.py b/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.py new file mode 100644 index 0000000000..491b9460e1 --- /dev/null +++ b/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.py @@ -0,0 +1,30 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: flyteidl/core/execution_envs.proto +"""Generated protocol buffer code.""" +from google.protobuf.internal import builder as _builder +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +from google.protobuf import struct_pb2 as google_dot_protobuf_dot_struct__pb2 + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\"flyteidl/core/execution_envs.proto\x12\rflyteidl.core\x1a\x1cgoogle/protobuf/struct.proto\"\x92\x01\n\x16\x45xecutionEnvAssignment\x12\x19\n\x08node_ids\x18\x01 \x03(\tR\x07nodeIds\x12\x1b\n\ttask_type\x18\x02 \x01(\tR\x08taskType\x12@\n\rexecution_env\x18\x03 \x01(\x0b\x32\x1b.flyteidl.core.ExecutionEnvR\x0c\x65xecutionEnv\"\xa3\x01\n\x0c\x45xecutionEnv\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n\x04type\x18\x02 \x01(\tR\x04type\x12\x31\n\x06\x65xtant\x18\x03 \x01(\x0b\x32\x17.google.protobuf.StructH\x00R\x06\x65xtant\x12-\n\x04spec\x18\x04 \x01(\x0b\x32\x17.google.protobuf.StructH\x00R\x04specB\r\n\x0b\x65nvironmentB\xb8\x01\n\x11\x63om.flyteidl.coreB\x12\x45xecutionEnvsProtoP\x01Z:github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core\xa2\x02\x03\x46\x43X\xaa\x02\rFlyteidl.Core\xca\x02\rFlyteidl\\Core\xe2\x02\x19\x46lyteidl\\Core\\GPBMetadata\xea\x02\x0e\x46lyteidl::Coreb\x06proto3') + +_globals = globals() +_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'flyteidl.core.execution_envs_pb2', _globals) +if _descriptor._USE_C_DESCRIPTORS == False: + + DESCRIPTOR._options = None + DESCRIPTOR._serialized_options = b'\n\021com.flyteidl.coreB\022ExecutionEnvsProtoP\001Z:github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core\242\002\003FCX\252\002\rFlyteidl.Core\312\002\rFlyteidl\\Core\342\002\031Flyteidl\\Core\\GPBMetadata\352\002\016Flyteidl::Core' + _globals['_EXECUTIONENVASSIGNMENT']._serialized_start=84 + _globals['_EXECUTIONENVASSIGNMENT']._serialized_end=230 + _globals['_EXECUTIONENV']._serialized_start=233 + _globals['_EXECUTIONENV']._serialized_end=396 +# @@protoc_insertion_point(module_scope) diff --git a/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.pyi b/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.pyi new file mode 100644 index 0000000000..1757c430af --- /dev/null +++ b/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2.pyi @@ -0,0 +1,29 @@ +from google.protobuf import struct_pb2 as _struct_pb2 +from google.protobuf.internal import containers as _containers +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union + +DESCRIPTOR: _descriptor.FileDescriptor + +class ExecutionEnvAssignment(_message.Message): + __slots__ = ["node_ids", "task_type", "execution_env"] + NODE_IDS_FIELD_NUMBER: _ClassVar[int] + TASK_TYPE_FIELD_NUMBER: _ClassVar[int] + EXECUTION_ENV_FIELD_NUMBER: _ClassVar[int] + node_ids: _containers.RepeatedScalarFieldContainer[str] + task_type: str + execution_env: ExecutionEnv + def __init__(self, node_ids: _Optional[_Iterable[str]] = ..., task_type: _Optional[str] = ..., execution_env: _Optional[_Union[ExecutionEnv, _Mapping]] = ...) -> None: ... + +class ExecutionEnv(_message.Message): + __slots__ = ["id", "type", "extant", "spec"] + ID_FIELD_NUMBER: _ClassVar[int] + TYPE_FIELD_NUMBER: _ClassVar[int] + EXTANT_FIELD_NUMBER: _ClassVar[int] + SPEC_FIELD_NUMBER: _ClassVar[int] + id: str + type: str + extant: _struct_pb2.Struct + spec: _struct_pb2.Struct + def __init__(self, id: _Optional[str] = ..., type: _Optional[str] = ..., extant: _Optional[_Union[_struct_pb2.Struct, _Mapping]] = ..., spec: _Optional[_Union[_struct_pb2.Struct, _Mapping]] = ...) -> None: ... diff --git a/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2_grpc.py b/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2_grpc.py new file mode 100644 index 0000000000..2daafffebf --- /dev/null +++ b/flyteidl/gen/pb_python/flyteidl/core/execution_envs_pb2_grpc.py @@ -0,0 +1,4 @@ +# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT! +"""Client and server classes corresponding to protobuf-defined services.""" +import grpc + diff --git a/flyteidl/gen/pb_rust/flyteidl.admin.rs b/flyteidl/gen/pb_rust/flyteidl.admin.rs index e402f1ad12..40c76c9d50 100644 --- a/flyteidl/gen/pb_rust/flyteidl.admin.rs +++ b/flyteidl/gen/pb_rust/flyteidl.admin.rs @@ -1409,6 +1409,9 @@ pub struct ExecutionSpec { /// Tags to be set for the execution. #[prost(string, repeated, tag="24")] pub tags: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + /// Execution environment assignments to be set for the execution. + #[prost(message, repeated, tag="25")] + pub execution_env_assignments: ::prost::alloc::vec::Vec, #[prost(oneof="execution_spec::NotificationOverrides", tags="5, 6")] pub notification_overrides: ::core::option::Option, } @@ -1821,6 +1824,9 @@ pub struct LaunchPlanSpec { /// Environment variables to be set for the execution. #[prost(message, optional, tag="21")] pub envs: ::core::option::Option, + /// Execution environment assignments to be set for the execution. + #[prost(message, repeated, tag="22")] + pub execution_env_assignments: ::prost::alloc::vec::Vec, } /// Values computed by the flyte platform after launch plan registration. /// These include expected_inputs required to be present in a CreateExecutionRequest @@ -2081,6 +2087,9 @@ pub struct WorkflowExecutionConfig { /// Environment variables to be set for the execution. #[prost(message, optional, tag="8")] pub envs: ::core::option::Option, + /// Execution environment assignments to be set for the execution. + #[prost(message, repeated, tag="9")] + pub execution_env_assignments: ::prost::alloc::vec::Vec, } /// Generic container for encapsulating all types of the above attributes messages. #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/flyteidl/gen/pb_rust/flyteidl.core.rs b/flyteidl/gen/pb_rust/flyteidl.core.rs index fba758fa3b..363d1d1db0 100644 --- a/flyteidl/gen/pb_rust/flyteidl.core.rs +++ b/flyteidl/gen/pb_rust/flyteidl.core.rs @@ -2934,6 +2934,50 @@ impl CatalogCacheStatus { } } } +/// ExecutionEnvAssignment is a message that is used to assign an execution environment to a set of +/// nodes. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ExecutionEnvAssignment { + /// node_ids is a list of node ids that are being assigned the execution environment. + #[prost(string, repeated, tag="1")] + pub node_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + /// task_type is the type of task that is being assigned. This is used to override which Flyte + /// plugin will be used during execution. + #[prost(string, tag="2")] + pub task_type: ::prost::alloc::string::String, + /// execution_env is the environment that is being assigned to the nodes. + #[prost(message, optional, tag="3")] + pub execution_env: ::core::option::Option, +} +/// ExecutionEnv is a message that is used to specify the execution environment. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ExecutionEnv { + /// id is a unique identifier for the execution environment. + #[prost(string, tag="1")] + pub id: ::prost::alloc::string::String, + /// type is the type of the execution environment. + #[prost(string, tag="2")] + pub r#type: ::prost::alloc::string::String, + /// environment is a oneof field that can be used to specify the environment in different ways. + #[prost(oneof="execution_env::Environment", tags="3, 4")] + pub environment: ::core::option::Option, +} +/// Nested message and enum types in `ExecutionEnv`. +pub mod execution_env { + /// environment is a oneof field that can be used to specify the environment in different ways. + #[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Environment { + /// extant is a reference to an existing environment. + #[prost(message, tag="3")] + Extant(::prost_types::Struct), + /// spec is a specification of the environment. + #[prost(message, tag="4")] + Spec(::prost_types::Struct), + } +} /// Describes a set of tasks to execute and how the final outputs are produced. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/flyteidl/protos/flyteidl/admin/execution.proto b/flyteidl/protos/flyteidl/admin/execution.proto index eea42b0ff3..eddf0dff02 100644 --- a/flyteidl/protos/flyteidl/admin/execution.proto +++ b/flyteidl/protos/flyteidl/admin/execution.proto @@ -7,6 +7,7 @@ import "flyteidl/admin/cluster_assignment.proto"; import "flyteidl/admin/common.proto"; import "flyteidl/core/literals.proto"; import "flyteidl/core/execution.proto"; +import "flyteidl/core/execution_envs.proto"; import "flyteidl/core/artifact_id.proto"; import "flyteidl/core/identifier.proto"; import "flyteidl/core/metrics.proto"; @@ -330,6 +331,9 @@ message ExecutionSpec { // Tags to be set for the execution. repeated string tags = 24; + + // Execution environment assignments to be set for the execution. + repeated core.ExecutionEnvAssignment execution_env_assignments = 25; } // Request to terminate an in-progress execution. This action is irreversible. diff --git a/flyteidl/protos/flyteidl/admin/launch_plan.proto b/flyteidl/protos/flyteidl/admin/launch_plan.proto index a13429f751..4be8dedb91 100644 --- a/flyteidl/protos/flyteidl/admin/launch_plan.proto +++ b/flyteidl/protos/flyteidl/admin/launch_plan.proto @@ -4,6 +4,7 @@ package flyteidl.admin; option go_package = "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin"; import "flyteidl/core/execution.proto"; +import "flyteidl/core/execution_envs.proto"; import "flyteidl/core/literals.proto"; import "flyteidl/core/identifier.proto"; import "flyteidl/core/interface.proto"; @@ -135,6 +136,9 @@ message LaunchPlanSpec { // Environment variables to be set for the execution. Envs envs = 21; + + // Execution environment assignments to be set for the execution. + repeated core.ExecutionEnvAssignment execution_env_assignments = 22; } // Values computed by the flyte platform after launch plan registration. diff --git a/flyteidl/protos/flyteidl/admin/matchable_resource.proto b/flyteidl/protos/flyteidl/admin/matchable_resource.proto index cca6dbcc41..1003700612 100644 --- a/flyteidl/protos/flyteidl/admin/matchable_resource.proto +++ b/flyteidl/protos/flyteidl/admin/matchable_resource.proto @@ -6,6 +6,7 @@ option go_package = "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin import "flyteidl/admin/common.proto"; import "flyteidl/admin/cluster_assignment.proto"; import "flyteidl/core/execution.proto"; +import "flyteidl/core/execution_envs.proto"; import "flyteidl/core/security.proto"; import "google/protobuf/wrappers.proto"; @@ -131,6 +132,9 @@ message WorkflowExecutionConfig { // Environment variables to be set for the execution. Envs envs = 8; + + // Execution environment assignments to be set for the execution. + repeated core.ExecutionEnvAssignment execution_env_assignments = 9; } // Generic container for encapsulating all types of the above attributes messages. diff --git a/flyteidl/protos/flyteidl/core/execution_envs.proto b/flyteidl/protos/flyteidl/core/execution_envs.proto new file mode 100644 index 0000000000..12aaa00dd5 --- /dev/null +++ b/flyteidl/protos/flyteidl/core/execution_envs.proto @@ -0,0 +1,39 @@ +syntax = "proto3"; + +package flyteidl.core; + +option go_package = "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core"; + +import "google/protobuf/struct.proto"; + +// ExecutionEnvAssignment is a message that is used to assign an execution environment to a set of +// nodes. +message ExecutionEnvAssignment { + // node_ids is a list of node ids that are being assigned the execution environment. + repeated string node_ids = 1; + + // task_type is the type of task that is being assigned. This is used to override which Flyte + // plugin will be used during execution. + string task_type = 2; + + // execution_env is the environment that is being assigned to the nodes. + ExecutionEnv execution_env = 3; +} + +// ExecutionEnv is a message that is used to specify the execution environment. +message ExecutionEnv { + // id is a unique identifier for the execution environment. + string id = 1; + + // type is the type of the execution environment. + string type = 2; + + // environment is a oneof field that can be used to specify the environment in different ways. + oneof environment { + // extant is a reference to an existing environment. + google.protobuf.Struct extant = 3; + + // spec is a specification of the environment. + google.protobuf.Struct spec = 4; + } +} diff --git a/flyteplugins/go/tasks/pluginmachinery/core/exec_context.go b/flyteplugins/go/tasks/pluginmachinery/core/exec_context.go index 11f48a33b2..e1a31d96c2 100644 --- a/flyteplugins/go/tasks/pluginmachinery/core/exec_context.go +++ b/flyteplugins/go/tasks/pluginmachinery/core/exec_context.go @@ -64,6 +64,9 @@ type TaskExecutionContext interface { // Returns a handle to the Task events recorder, which get stored in the Admin. EventsRecorder() EventsRecorder + + // GetExecutionEnvClient returns the execution environment client. + GetExecutionEnvClient() ExecutionEnvClient } // A simple fire-and-forget func diff --git a/flyteplugins/go/tasks/pluginmachinery/core/execution_env.go b/flyteplugins/go/tasks/pluginmachinery/core/execution_env.go new file mode 100644 index 0000000000..3a7e4dac62 --- /dev/null +++ b/flyteplugins/go/tasks/pluginmachinery/core/execution_env.go @@ -0,0 +1,14 @@ +package core + +import ( + "context" + + _struct "github.com/golang/protobuf/ptypes/struct" +) + +// ExecutionEnvClient is an interface that defines the methods to interact with an execution +// environment. +type ExecutionEnvClient interface { + Get(ctx context.Context, executionEnvID string) *_struct.Struct + Create(ctx context.Context, executionEnvID string, executionEnvSpec *_struct.Struct) (*_struct.Struct, error) +} diff --git a/flyteplugins/go/tasks/pluginmachinery/core/mocks/execution_env_client.go b/flyteplugins/go/tasks/pluginmachinery/core/mocks/execution_env_client.go new file mode 100644 index 0000000000..c7436d3059 --- /dev/null +++ b/flyteplugins/go/tasks/pluginmachinery/core/mocks/execution_env_client.go @@ -0,0 +1,91 @@ +// Code generated by mockery v1.0.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + mock "github.com/stretchr/testify/mock" + + structpb "google.golang.org/protobuf/types/known/structpb" +) + +// ExecutionEnvClient is an autogenerated mock type for the ExecutionEnvClient type +type ExecutionEnvClient struct { + mock.Mock +} + +type ExecutionEnvClient_Create struct { + *mock.Call +} + +func (_m ExecutionEnvClient_Create) Return(_a0 *structpb.Struct, _a1 error) *ExecutionEnvClient_Create { + return &ExecutionEnvClient_Create{Call: _m.Call.Return(_a0, _a1)} +} + +func (_m *ExecutionEnvClient) OnCreate(ctx context.Context, executionEnvID string, executionEnvSpec *structpb.Struct) *ExecutionEnvClient_Create { + c_call := _m.On("Create", ctx, executionEnvID, executionEnvSpec) + return &ExecutionEnvClient_Create{Call: c_call} +} + +func (_m *ExecutionEnvClient) OnCreateMatch(matchers ...interface{}) *ExecutionEnvClient_Create { + c_call := _m.On("Create", matchers...) + return &ExecutionEnvClient_Create{Call: c_call} +} + +// Create provides a mock function with given fields: ctx, executionEnvID, executionEnvSpec +func (_m *ExecutionEnvClient) Create(ctx context.Context, executionEnvID string, executionEnvSpec *structpb.Struct) (*structpb.Struct, error) { + ret := _m.Called(ctx, executionEnvID, executionEnvSpec) + + var r0 *structpb.Struct + if rf, ok := ret.Get(0).(func(context.Context, string, *structpb.Struct) *structpb.Struct); ok { + r0 = rf(ctx, executionEnvID, executionEnvSpec) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*structpb.Struct) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, *structpb.Struct) error); ok { + r1 = rf(ctx, executionEnvID, executionEnvSpec) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type ExecutionEnvClient_Get struct { + *mock.Call +} + +func (_m ExecutionEnvClient_Get) Return(_a0 *structpb.Struct) *ExecutionEnvClient_Get { + return &ExecutionEnvClient_Get{Call: _m.Call.Return(_a0)} +} + +func (_m *ExecutionEnvClient) OnGet(ctx context.Context, executionEnvID string) *ExecutionEnvClient_Get { + c_call := _m.On("Get", ctx, executionEnvID) + return &ExecutionEnvClient_Get{Call: c_call} +} + +func (_m *ExecutionEnvClient) OnGetMatch(matchers ...interface{}) *ExecutionEnvClient_Get { + c_call := _m.On("Get", matchers...) + return &ExecutionEnvClient_Get{Call: c_call} +} + +// Get provides a mock function with given fields: ctx, executionEnvID +func (_m *ExecutionEnvClient) Get(ctx context.Context, executionEnvID string) *structpb.Struct { + ret := _m.Called(ctx, executionEnvID) + + var r0 *structpb.Struct + if rf, ok := ret.Get(0).(func(context.Context, string) *structpb.Struct); ok { + r0 = rf(ctx, executionEnvID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*structpb.Struct) + } + } + + return r0 +} diff --git a/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_execution_context.go b/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_execution_context.go index ec4b5e346f..19f144e9df 100644 --- a/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_execution_context.go +++ b/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_execution_context.go @@ -120,6 +120,40 @@ func (_m *TaskExecutionContext) EventsRecorder() core.EventsRecorder { return r0 } +type TaskExecutionContext_GetExecutionEnvClient struct { + *mock.Call +} + +func (_m TaskExecutionContext_GetExecutionEnvClient) Return(_a0 core.ExecutionEnvClient) *TaskExecutionContext_GetExecutionEnvClient { + return &TaskExecutionContext_GetExecutionEnvClient{Call: _m.Call.Return(_a0)} +} + +func (_m *TaskExecutionContext) OnGetExecutionEnvClient() *TaskExecutionContext_GetExecutionEnvClient { + c_call := _m.On("GetExecutionEnvClient") + return &TaskExecutionContext_GetExecutionEnvClient{Call: c_call} +} + +func (_m *TaskExecutionContext) OnGetExecutionEnvClientMatch(matchers ...interface{}) *TaskExecutionContext_GetExecutionEnvClient { + c_call := _m.On("GetExecutionEnvClient", matchers...) + return &TaskExecutionContext_GetExecutionEnvClient{Call: c_call} +} + +// GetExecutionEnvClient provides a mock function with given fields: +func (_m *TaskExecutionContext) GetExecutionEnvClient() core.ExecutionEnvClient { + ret := _m.Called() + + var r0 core.ExecutionEnvClient + if rf, ok := ret.Get(0).(func() core.ExecutionEnvClient); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(core.ExecutionEnvClient) + } + } + + return r0 +} + type TaskExecutionContext_InputReader struct { *mock.Call } diff --git a/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_overrides.go b/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_overrides.go index 57c9f9db2d..d00973c114 100644 --- a/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_overrides.go +++ b/flyteplugins/go/tasks/pluginmachinery/core/mocks/task_overrides.go @@ -82,40 +82,6 @@ func (_m *TaskOverrides) GetExtendedResources() *flyteidlcore.ExtendedResources return r0 } -type TaskOverrides_GetContainerImage struct { - *mock.Call -} - -func (_m TaskOverrides_GetContainerImage) Return(_a0 string) *TaskOverrides_GetContainerImage { - return &TaskOverrides_GetContainerImage{Call: _m.Call.Return(_a0)} -} - -func (_m *TaskOverrides) OnGetContainerImage() *TaskOverrides_GetContainerImage { - c_call := _m.On("GetContainerImage") - return &TaskOverrides_GetContainerImage{Call: c_call} -} - -func (_m *TaskOverrides) OnGetContainerImageMatch(matchers ...interface{}) *TaskOverrides_GetContainerImage { - c_call := _m.On("GetContainerImage", matchers...) - return &TaskOverrides_GetContainerImage{Call: c_call} -} - -// GetContainerImage provides a mock function with given fields: -func (_m *TaskOverrides) GetContainerImage() string { - ret := _m.Called() - - var r0 string - if rf, ok := ret.Get(0).(func() string); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(string) - } - } - - return r0 -} - type TaskOverrides_GetResources struct { *mock.Call } diff --git a/flyteplugins/go/tasks/plugins/k8s/ray/config_flags.go b/flyteplugins/go/tasks/plugins/k8s/ray/config_flags.go index 8113a26278..398ca419eb 100755 --- a/flyteplugins/go/tasks/plugins/k8s/ray/config_flags.go +++ b/flyteplugins/go/tasks/plugins/k8s/ray/config_flags.go @@ -59,5 +59,6 @@ func (cfg Config) GetPFlagSet(prefix string) *pflag.FlagSet { cmdFlags.String(fmt.Sprintf("%v%v", prefix, "remoteClusterConfig.endpoint"), defaultConfig.RemoteClusterConfig.Endpoint, " Remote K8s cluster endpoint") cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "remoteClusterConfig.enabled"), defaultConfig.RemoteClusterConfig.Enabled, " Boolean flag to enable or disable") cmdFlags.Bool(fmt.Sprintf("%v%v", prefix, "enableUsageStats"), defaultConfig.EnableUsageStats, "Enable usage stats for ray jobs. These stats are submitted to usage-stats.ray.io per https://docs.ray.io/en/latest/cluster/usage-stats.html") + cmdFlags.String(fmt.Sprintf("%v%v", prefix, "kubeRayCrdVersion"), defaultConfig.KubeRayCrdVersion, "Version of the Ray CRD to use when creating RayClusters or RayJobs.") return cmdFlags } diff --git a/flyteplugins/go/tasks/plugins/k8s/ray/config_flags_test.go b/flyteplugins/go/tasks/plugins/k8s/ray/config_flags_test.go index f05c62c8e1..820846c17a 100755 --- a/flyteplugins/go/tasks/plugins/k8s/ray/config_flags_test.go +++ b/flyteplugins/go/tasks/plugins/k8s/ray/config_flags_test.go @@ -225,4 +225,18 @@ func TestConfig_SetFlags(t *testing.T) { } }) }) + t.Run("Test_kubeRayCrdVersion", func(t *testing.T) { + + t.Run("Override", func(t *testing.T) { + testValue := "1" + + cmdFlags.Set("kubeRayCrdVersion", testValue) + if vString, err := cmdFlags.GetString("kubeRayCrdVersion"); err == nil { + testDecodeJson_Config(t, fmt.Sprintf("%v", vString), &actual.KubeRayCrdVersion) + + } else { + assert.FailNow(t, err.Error()) + } + }) + }) } diff --git a/flytepropeller/go.mod b/flytepropeller/go.mod index bcb3498ee0..14566a9270 100644 --- a/flytepropeller/go.mod +++ b/flytepropeller/go.mod @@ -30,6 +30,7 @@ require ( github.com/spf13/cobra v1.7.0 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.8.4 + github.com/unionai/flyte/fasttask/plugin v0.0.0-00010101000000-000000000000 go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 go.opentelemetry.io/otel v1.24.0 go.opentelemetry.io/otel/trace v1.24.0 @@ -167,6 +168,7 @@ replace ( github.com/flyteorg/flyte/flyteidl => ../flyteidl github.com/flyteorg/flyte/flyteplugins => ../flyteplugins github.com/flyteorg/flyte/flytestdlib => ../flytestdlib + github.com/unionai/flyte/fasttask/plugin => ../fasttask/plugin k8s.io/api => k8s.io/api v0.28.2 k8s.io/apimachinery => k8s.io/apimachinery v0.28.2 k8s.io/client-go => k8s.io/client-go v0.28.2 diff --git a/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/execution_config.go b/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/execution_config.go index 30cd9fa0de..77f3042eb8 100644 --- a/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/execution_config.go +++ b/flytepropeller/pkg/apis/flyteworkflow/v1alpha1/execution_config.go @@ -1,9 +1,13 @@ package v1alpha1 import ( + "bytes" + + "github.com/golang/protobuf/jsonpb" "k8s.io/apimachinery/pkg/api/resource" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/admin" + "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" ) // This contains an OutputLocationPrefix. When running against AWS, this should be something of the form @@ -35,6 +39,8 @@ type ExecutionConfig struct { OverwriteCache bool // Defines a map of environment variable name / value pairs that are applied to all tasks. EnvironmentVariables map[string]string + // ExecutionEnvAssignments defines execution environment assignments to be set for the execution. + ExecutionEnvAssignments []ExecutionEnvAssignment } type TaskPluginOverride struct { @@ -59,3 +65,22 @@ type TaskResources struct { // A hard limit, a task cannot consume resources greater than the limit specifies. Limits TaskResourceSpec } + +// ExecutionEnvAssignment is a wrapper around core.ExecutionEnvAssignment to define +// and assign an execution environment to a collection of workflow nodes. +type ExecutionEnvAssignment struct { + *core.ExecutionEnvAssignment +} + +func (in *ExecutionEnvAssignment) MarshalJSON() ([]byte, error) { + var buf bytes.Buffer + if err := marshaler.Marshal(&buf, in.ExecutionEnvAssignment); err != nil { + return nil, err + } + return buf.Bytes(), nil +} + +func (in *ExecutionEnvAssignment) UnmarshalJSON(b []byte) error { + in.ExecutionEnvAssignment = &core.ExecutionEnvAssignment{} + return jsonpb.Unmarshal(bytes.NewReader(b), in.ExecutionEnvAssignment) +} diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/002_core.containerization.multi_images.my_workflow_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/002_core.containerization.multi_images.my_workflow_2_wf_crd.json index bdee75fb40..9012665e92 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/002_core.containerization.multi_images.my_workflow_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/002_core.containerization.multi_images.my_workflow_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-containerization-multi-images-my-workflow"}},"spec":{"id":"::core.containerization.multi_images.my_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"svm_trainer","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.multi_images.svm_trainer\""},"n1":{"id":"n1","name":"svm_predictor","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.multi_images.svm_predictor\"","inputBindings":[{"var":"X_test","binding":{"promise":{"nodeId":"n0","var":"test_features"}}},{"var":"X_train","binding":{"promise":{"nodeId":"n0","var":"train_features"}}},{"var":"y_test","binding":{"promise":{"nodeId":"n0","var":"test_labels"}}},{"var":"y_train","binding":{"promise":{"nodeId":"n0","var":"train_labels"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.containerization.multi_images.my_workflow"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.containerization.multi_images.svm_predictor\"":{"id":{"resourceType":"TASK","name":"core.containerization.multi_images.svm_predictor"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"X_test":{"type":{"structuredDatasetType":{"format":"parquet"}}},"X_train":{"type":{"structuredDatasetType":{"format":"parquet"}}},"y_test":{"type":{"structuredDatasetType":{"format":"parquet"}}},"y_train":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:multi-image-predict-98b125fd57d20594026941c2ebe7ef662e5acb7d6423660a65f493ca2d9aa267","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.multi_images","task-name","svm_predictor"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.containerization.multi_images.svm_trainer\"":{"id":{"resourceType":"TASK","name":"core.containerization.multi_images.svm_trainer"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"test_features":{"type":{"structuredDatasetType":{"format":"parquet"}}},"test_labels":{"type":{"structuredDatasetType":{"format":"parquet"}}},"train_features":{"type":{"structuredDatasetType":{"format":"parquet"}}},"train_labels":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-with-sklearn-baa17ccf39aa667c5950bd713a4366ce7d5fccaf7f85e6be8c07fe4b522f92c3","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.multi_images","task-name","svm_trainer"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-containerization-multi-images-my-workflow"}},"spec":{"id":"::core.containerization.multi_images.my_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"svm_trainer","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.multi_images.svm_trainer\""},"n1":{"id":"n1","name":"svm_predictor","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.multi_images.svm_predictor\"","inputBindings":[{"var":"X_test","binding":{"promise":{"nodeId":"n0","var":"test_features"}}},{"var":"X_train","binding":{"promise":{"nodeId":"n0","var":"train_features"}}},{"var":"y_test","binding":{"promise":{"nodeId":"n0","var":"test_labels"}}},{"var":"y_train","binding":{"promise":{"nodeId":"n0","var":"train_labels"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.containerization.multi_images.my_workflow"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.containerization.multi_images.svm_predictor\"":{"id":{"resourceType":"TASK","name":"core.containerization.multi_images.svm_predictor"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"X_test":{"type":{"structuredDatasetType":{"format":"parquet"}}},"X_train":{"type":{"structuredDatasetType":{"format":"parquet"}}},"y_test":{"type":{"structuredDatasetType":{"format":"parquet"}}},"y_train":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:multi-image-predict-98b125fd57d20594026941c2ebe7ef662e5acb7d6423660a65f493ca2d9aa267","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.multi_images","task-name","svm_predictor"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.containerization.multi_images.svm_trainer\"":{"id":{"resourceType":"TASK","name":"core.containerization.multi_images.svm_trainer"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"test_features":{"type":{"structuredDatasetType":{"format":"parquet"}}},"test_labels":{"type":{"structuredDatasetType":{"format":"parquet"}}},"train_features":{"type":{"structuredDatasetType":{"format":"parquet"}}},"train_labels":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-with-sklearn-baa17ccf39aa667c5950bd713a4366ce7d5fccaf7f85e6be8c07fe4b522f92c3","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.multi_images","task-name","svm_trainer"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/010_core.containerization.raw_container.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/010_core.containerization.raw_container.wf_2_wf_crd.json index d22a6eb2f1..d691c24928 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/010_core.containerization.raw_container.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/010_core.containerization.raw_container.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-containerization-raw-container-wf"}},"spec":{"id":"::core.containerization.raw_container.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"ellipse-area-metadata-shell","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-shell\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n1":{"id":"n1","name":"ellipse-area-metadata-python","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-python\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n2":{"id":"n2","name":"ellipse-area-metadata-r","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-r\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n3":{"id":"n3","name":"ellipse-area-metadata-haskell","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-haskell\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n4":{"id":"n4","name":"ellipse-area-metadata-julia","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-julia\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n5":{"id":"n5","name":"report_all_calculated_areas","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.raw_container.report_all_calculated_areas\"","inputBindings":[{"var":"area_haskell","binding":{"promise":{"nodeId":"n3","var":"area"}}},{"var":"area_julia","binding":{"promise":{"nodeId":"n4","var":"area"}}},{"var":"area_python","binding":{"promise":{"nodeId":"n1","var":"area"}}},{"var":"area_r","binding":{"promise":{"nodeId":"n2","var":"area"}}},{"var":"area_shell","binding":{"promise":{"nodeId":"n0","var":"area"}}},{"var":"metadata_haskell","binding":{"promise":{"nodeId":"n3","var":"metadata"}}},{"var":"metadata_julia","binding":{"promise":{"nodeId":"n4","var":"metadata"}}},{"var":"metadata_python","binding":{"promise":{"nodeId":"n1","var":"metadata"}}},{"var":"metadata_r","binding":{"promise":{"nodeId":"n2","var":"metadata"}}},{"var":"metadata_shell","binding":{"promise":{"nodeId":"n0","var":"metadata"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n5"],"n1":["n5"],"n2":["n5"],"n3":["n5"],"n4":["n5"],"n5":["end-node"],"start-node":["n0","n1","n2","n3","n4"]},"edges":{"downstream":{"n0":["n5"],"n1":["n5"],"n2":["n5"],"n3":["n5"],"n4":["n5"],"n5":["end-node"],"start-node":["n0","n1","n2","n3","n4"]},"upstream":{"end-node":["n5"],"n0":["start-node"],"n1":["start-node"],"n2":["start-node"],"n3":["start-node"],"n4":["start-node"],"n5":["n0","n1","n2","n3","n4"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.containerization.raw_container.wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"floatValue":0}}},"b":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.containerization.raw_container.report_all_calculated_areas\"":{"id":{"resourceType":"TASK","name":"core.containerization.raw_container.report_all_calculated_areas"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"area_haskell":{"type":{"simple":"FLOAT"}},"area_julia":{"type":{"simple":"FLOAT"}},"area_python":{"type":{"simple":"FLOAT"}},"area_r":{"type":{"simple":"FLOAT"}},"area_shell":{"type":{"simple":"FLOAT"}},"metadata_haskell":{"type":{"simple":"STRING"}},"metadata_julia":{"type":{"simple":"STRING"}},"metadata_python":{"type":{"simple":"STRING"}},"metadata_r":{"type":{"simple":"STRING"}},"metadata_shell":{"type":{"simple":"STRING"}}}},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.raw_container","task-name","report_all_calculated_areas"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"ellipse-area-metadata-haskell\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-haskell"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-haskell:v1","command":["./calculate-ellipse-area","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-julia\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-julia"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-julia:v1","command":["julia","calculate-ellipse-area.jl","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-python\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-python"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-python:v1","command":["python","calculate-ellipse-area.py","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-r\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-r"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-r:v1","command":["Rscript","--vanilla","calculate-ellipse-area.R","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-shell\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-shell"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-shell:v1","command":["./calculate-ellipse-area.sh","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-containerization-raw-container-wf"}},"spec":{"id":"::core.containerization.raw_container.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"ellipse-area-metadata-shell","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-shell\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n1":{"id":"n1","name":"ellipse-area-metadata-python","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-python\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n2":{"id":"n2","name":"ellipse-area-metadata-r","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-r\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n3":{"id":"n3","name":"ellipse-area-metadata-haskell","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-haskell\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n4":{"id":"n4","name":"ellipse-area-metadata-julia","resources":{},"kind":"task","task":"resource_type:TASK name:\"ellipse-area-metadata-julia\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n5":{"id":"n5","name":"report_all_calculated_areas","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.raw_container.report_all_calculated_areas\"","inputBindings":[{"var":"area_haskell","binding":{"promise":{"nodeId":"n3","var":"area"}}},{"var":"area_julia","binding":{"promise":{"nodeId":"n4","var":"area"}}},{"var":"area_python","binding":{"promise":{"nodeId":"n1","var":"area"}}},{"var":"area_r","binding":{"promise":{"nodeId":"n2","var":"area"}}},{"var":"area_shell","binding":{"promise":{"nodeId":"n0","var":"area"}}},{"var":"metadata_haskell","binding":{"promise":{"nodeId":"n3","var":"metadata"}}},{"var":"metadata_julia","binding":{"promise":{"nodeId":"n4","var":"metadata"}}},{"var":"metadata_python","binding":{"promise":{"nodeId":"n1","var":"metadata"}}},{"var":"metadata_r","binding":{"promise":{"nodeId":"n2","var":"metadata"}}},{"var":"metadata_shell","binding":{"promise":{"nodeId":"n0","var":"metadata"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n5"],"n1":["n5"],"n2":["n5"],"n3":["n5"],"n4":["n5"],"n5":["end-node"],"start-node":["n0","n1","n2","n3","n4"]},"edges":{"downstream":{"n0":["n5"],"n1":["n5"],"n2":["n5"],"n3":["n5"],"n4":["n5"],"n5":["end-node"],"start-node":["n0","n1","n2","n3","n4"]},"upstream":{"end-node":["n5"],"n0":["start-node"],"n1":["start-node"],"n2":["start-node"],"n3":["start-node"],"n4":["start-node"],"n5":["n0","n1","n2","n3","n4"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.containerization.raw_container.wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"floatValue":0}}},"b":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.containerization.raw_container.report_all_calculated_areas\"":{"id":{"resourceType":"TASK","name":"core.containerization.raw_container.report_all_calculated_areas"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"area_haskell":{"type":{"simple":"FLOAT"}},"area_julia":{"type":{"simple":"FLOAT"}},"area_python":{"type":{"simple":"FLOAT"}},"area_r":{"type":{"simple":"FLOAT"}},"area_shell":{"type":{"simple":"FLOAT"}},"metadata_haskell":{"type":{"simple":"STRING"}},"metadata_julia":{"type":{"simple":"STRING"}},"metadata_python":{"type":{"simple":"STRING"}},"metadata_r":{"type":{"simple":"STRING"}},"metadata_shell":{"type":{"simple":"STRING"}}}},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.raw_container","task-name","report_all_calculated_areas"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"ellipse-area-metadata-haskell\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-haskell"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-haskell:v1","command":["./calculate-ellipse-area","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-julia\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-julia"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-julia:v1","command":["julia","calculate-ellipse-area.jl","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-python\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-python"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-python:v1","command":["python","calculate-ellipse-area.py","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-r\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-r"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-r:v1","command":["Rscript","--vanilla","calculate-ellipse-area.R","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}},"resource_type:TASK name:\"ellipse-area-metadata-shell\"":{"id":{"resourceType":"TASK","name":"ellipse-area-metadata-shell"},"type":"raw-container","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"area":{"type":{"simple":"FLOAT"}},"metadata":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/rawcontainers-shell:v1","command":["./calculate-ellipse-area.sh","/var/inputs","/var/outputs"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}],"dataConfig":{"enabled":true,"inputPath":"/var/inputs","outputPath":"/var/outputs"}}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/015_core.containerization.use_secrets.my_secret_workflow_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/015_core.containerization.use_secrets.my_secret_workflow_2_wf_crd.json index 19d773cb50..f64c095d4f 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/015_core.containerization.use_secrets.my_secret_workflow_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/015_core.containerization.use_secrets.my_secret_workflow_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-containerization-use-secrets-my-secret-workflow"}},"spec":{"id":"::core.containerization.use_secrets.my_secret_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o2","binding":{"promise":{"nodeId":"n1","var":"o1"}}},{"var":"o3","binding":{"promise":{"nodeId":"n2","var":"o0"}}},{"var":"o4","binding":{"promise":{"nodeId":"n2","var":"o1"}}}]},"n0":{"id":"n0","name":"secret_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.use_secrets.secret_task\""},"n1":{"id":"n1","name":"user_info_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.use_secrets.user_info_task\""},"n2":{"id":"n2","name":"secret_file_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.use_secrets.secret_file_task\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"edges":{"downstream":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"upstream":{"end-node":["n0","n1","n2"],"n0":["start-node"],"n1":["start-node"],"n2":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o2","binding":{"promise":{"nodeId":"n1","var":"o1"}}},{"var":"o3","binding":{"promise":{"nodeId":"n2","var":"o0"}}},{"var":"o4","binding":{"promise":{"nodeId":"n2","var":"o1"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.containerization.use_secrets.my_secret_workflow"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}},"o2":{"type":{"simple":"STRING"}},"o3":{"type":{"simple":"STRING"}},"o4":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.containerization.use_secrets.secret_file_task\"":{"id":{"resourceType":"TASK","name":"core.containerization.use_secrets.secret_file_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.use_secrets","task-name","secret_file_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"securityContext":{"secrets":[{"group":"user-info","key":"user_secret","mountRequirement":"ENV_VAR"}]}},"resource_type:TASK name:\"core.containerization.use_secrets.secret_task\"":{"id":{"resourceType":"TASK","name":"core.containerization.use_secrets.secret_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.use_secrets","task-name","secret_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"securityContext":{"secrets":[{"group":"user-info","key":"user_secret"}]}},"resource_type:TASK name:\"core.containerization.use_secrets.user_info_task\"":{"id":{"resourceType":"TASK","name":"core.containerization.use_secrets.user_info_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.use_secrets","task-name","user_info_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"securityContext":{"secrets":[{"group":"user-info","key":"username"},{"group":"user-info","key":"password"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-containerization-use-secrets-my-secret-workflow"}},"spec":{"id":"::core.containerization.use_secrets.my_secret_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o2","binding":{"promise":{"nodeId":"n1","var":"o1"}}},{"var":"o3","binding":{"promise":{"nodeId":"n2","var":"o0"}}},{"var":"o4","binding":{"promise":{"nodeId":"n2","var":"o1"}}}]},"n0":{"id":"n0","name":"secret_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.use_secrets.secret_task\""},"n1":{"id":"n1","name":"user_info_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.use_secrets.user_info_task\""},"n2":{"id":"n2","name":"secret_file_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.containerization.use_secrets.secret_file_task\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"edges":{"downstream":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"upstream":{"end-node":["n0","n1","n2"],"n0":["start-node"],"n1":["start-node"],"n2":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o2","binding":{"promise":{"nodeId":"n1","var":"o1"}}},{"var":"o3","binding":{"promise":{"nodeId":"n2","var":"o0"}}},{"var":"o4","binding":{"promise":{"nodeId":"n2","var":"o1"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.containerization.use_secrets.my_secret_workflow"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}},"o2":{"type":{"simple":"STRING"}},"o3":{"type":{"simple":"STRING"}},"o4":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.containerization.use_secrets.secret_file_task\"":{"id":{"resourceType":"TASK","name":"core.containerization.use_secrets.secret_file_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.use_secrets","task-name","secret_file_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"securityContext":{"secrets":[{"group":"user-info","key":"user_secret","mountRequirement":"ENV_VAR"}]}},"resource_type:TASK name:\"core.containerization.use_secrets.secret_task\"":{"id":{"resourceType":"TASK","name":"core.containerization.use_secrets.secret_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.use_secrets","task-name","secret_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"securityContext":{"secrets":[{"group":"user-info","key":"user_secret"}]}},"resource_type:TASK name:\"core.containerization.use_secrets.user_info_task\"":{"id":{"resourceType":"TASK","name":"core.containerization.use_secrets.user_info_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.containerization.use_secrets","task-name","user_info_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"securityContext":{"secrets":[{"group":"user-info","key":"username"},{"group":"user-info","key":"password"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/019_core.control_flow.chain_tasks.chain_tasks_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/019_core.control_flow.chain_tasks.chain_tasks_wf_2_wf_crd.json index a191977c87..1518156400 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/019_core.control_flow.chain_tasks.chain_tasks_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/019_core.control_flow.chain_tasks.chain_tasks_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-chain-tasks-chain-tasks-wf"}},"spec":{"id":"::core.control_flow.chain_tasks.chain_tasks_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"write","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.chain_tasks.write\""},"n1":{"id":"n1","name":"read","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.chain_tasks.read\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.chain_tasks.chain_tasks_wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.chain_tasks.read\"":{"id":{"resourceType":"TASK","name":"core.control_flow.chain_tasks.read"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.chain_tasks","task-name","read"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.chain_tasks.write\"":{"id":{"resourceType":"TASK","name":"core.control_flow.chain_tasks.write"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.chain_tasks","task-name","write"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-chain-tasks-chain-tasks-wf"}},"spec":{"id":"::core.control_flow.chain_tasks.chain_tasks_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"write","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.chain_tasks.write\""},"n1":{"id":"n1","name":"read","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.chain_tasks.read\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.chain_tasks.chain_tasks_wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.chain_tasks.read\"":{"id":{"resourceType":"TASK","name":"core.control_flow.chain_tasks.read"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.chain_tasks","task-name","read"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.chain_tasks.write\"":{"id":{"resourceType":"TASK","name":"core.control_flow.chain_tasks.write"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.chain_tasks","task-name","write"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/022_core.control_flow.checkpoint.example_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/022_core.control_flow.checkpoint.example_2_wf_crd.json index a0dbf22966..1ccc4f641c 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/022_core.control_flow.checkpoint.example_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/022_core.control_flow.checkpoint.example_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-checkpoint-example"}},"spec":{"id":"::core.control_flow.checkpoint.example","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"use_checkpoint","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.checkpoint.use_checkpoint\"","inputBindings":[{"var":"n_iterations","binding":{"promise":{"nodeId":"start-node","var":"n_iterations"}}}],"retry":{"minAttempts":4}},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.checkpoint.example"},"interface":{"inputs":{"variables":{"n_iterations":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"n_iterations":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.checkpoint.use_checkpoint\"":{"id":{"resourceType":"TASK","name":"core.control_flow.checkpoint.use_checkpoint"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{"retries":3}},"interface":{"inputs":{"variables":{"n_iterations":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.checkpoint","task-name","use_checkpoint"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-checkpoint-example"}},"spec":{"id":"::core.control_flow.checkpoint.example","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"use_checkpoint","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.checkpoint.use_checkpoint\"","inputBindings":[{"var":"n_iterations","binding":{"promise":{"nodeId":"start-node","var":"n_iterations"}}}],"retry":{"minAttempts":4}},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.checkpoint.example"},"interface":{"inputs":{"variables":{"n_iterations":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"n_iterations":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.checkpoint.use_checkpoint\"":{"id":{"resourceType":"TASK","name":"core.control_flow.checkpoint.use_checkpoint"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{"retries":3}},"interface":{"inputs":{"variables":{"n_iterations":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.checkpoint","task-name","use_checkpoint"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/026_core.control_flow.conditions.multiplier_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/026_core.control_flow.conditions.multiplier_2_wf_crd.json index fbe6673294..9d64554440 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/026_core.control_flow.conditions.multiplier_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/026_core.control_flow.conditions.multiplier_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-multiplier"}},"spec":{"id":"::core.control_flow.conditions.multiplier","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GTE","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LTE","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"else":"n0-n1"},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.multiplier"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-multiplier"}},"spec":{"id":"::core.control_flow.conditions.multiplier","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GTE","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LTE","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"else":"n0-n1"},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.multiplier"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/028_core.control_flow.conditions.multiplier_2_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/028_core.control_flow.conditions.multiplier_2_2_wf_crd.json index b38f23532a..d8c95f33f8 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/028_core.control_flow.conditions.multiplier_2_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/028_core.control_flow.conditions.multiplier_2_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-multiplier-2"}},"spec":{"id":"::core.control_flow.conditions.multiplier_2","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}},"rightExpression":{"comparison":{"operator":"LTE","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":10}}}}}},"then":"n0-n1"}],"elseFail":{"failed_node_id":"fractions","message":"The input must be between 0 and 10"}},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.multiplier_2"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-multiplier-2"}},"spec":{"id":"::core.control_flow.conditions.multiplier_2","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}},"rightExpression":{"comparison":{"operator":"LTE","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":10}}}}}},"then":"n0-n1"}],"elseFail":{"failed_node_id":"fractions","message":"The input must be between 0 and 10"}},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.multiplier_2"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/030_core.control_flow.conditions.multiplier_3_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/030_core.control_flow.conditions.multiplier_3_2_wf_crd.json index f5835c6d5d..499c9c2033 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/030_core.control_flow.conditions.multiplier_3_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/030_core.control_flow.conditions.multiplier_3_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-multiplier-3"}},"spec":{"id":"::core.control_flow.conditions.multiplier_3","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":10}}}}}},"then":"n0-n1"}],"elseFail":{"failed_node_id":"fractions","message":"The input must be between 0 and 10"}},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n1":{"id":"n1","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.multiplier_3"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-multiplier-3"}},"spec":{"id":"::core.control_flow.conditions.multiplier_3","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":10}}}}}},"then":"n0-n1"}],"elseFail":{"failed_node_id":"fractions","message":"The input must be between 0 and 10"}},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n1":{"id":"n1","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.multiplier_3"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/035_core.control_flow.conditions.basic_boolean_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/035_core.control_flow.conditions.basic_boolean_wf_2_wf_crd.json index f409d9da6d..4df757b976 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/035_core.control_flow.conditions.basic_boolean_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/035_core.control_flow.conditions.basic_boolean_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-basic-boolean-wf"}},"spec":{"id":"::core.control_flow.conditions.basic_boolean_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"coin_toss","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"","inputBindings":[{"var":"seed","binding":{"promise":{"nodeId":"start-node","var":"seed"}}}]},"n1":{"id":"n1","name":"test","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"leftValue":{"var":"n0.o0"},"rightValue":{"primitive":{"boolean":true}}}},"then":"n1-n0"},"else":"n1-n1"},"inputBindings":[{"var":"n0.o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n1-n0":{"id":"n1-n0","name":"success","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.success\""},"n1-n1":{"id":"n1-n1","name":"failed","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.failed\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.basic_boolean_wf"},"interface":{"inputs":{"variables":{"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"seed":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.coin_toss"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"BOOLEAN"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","coin_toss"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.failed\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.failed"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","failed"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.success\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.success"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","success"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-basic-boolean-wf"}},"spec":{"id":"::core.control_flow.conditions.basic_boolean_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"coin_toss","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"","inputBindings":[{"var":"seed","binding":{"promise":{"nodeId":"start-node","var":"seed"}}}]},"n1":{"id":"n1","name":"test","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"leftValue":{"var":"n0.o0"},"rightValue":{"primitive":{"boolean":true}}}},"then":"n1-n0"},"else":"n1-n1"},"inputBindings":[{"var":"n0.o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n1-n0":{"id":"n1-n0","name":"success","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.success\""},"n1-n1":{"id":"n1-n1","name":"failed","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.failed\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.basic_boolean_wf"},"interface":{"inputs":{"variables":{"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"seed":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.coin_toss"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"BOOLEAN"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","coin_toss"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.failed\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.failed"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","failed"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.success\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.success"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","success"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/037_core.control_flow.conditions.bool_input_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/037_core.control_flow.conditions.bool_input_wf_2_wf_crd.json index 65ceb0f108..c68d5772d1 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/037_core.control_flow.conditions.bool_input_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/037_core.control_flow.conditions.bool_input_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-bool-input-wf"}},"spec":{"id":"::core.control_flow.conditions.bool_input_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"test","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"leftValue":{"var":".b"},"rightValue":{"primitive":{"boolean":true}}}},"then":"n0-n0"},"else":"n0-n1"},"inputBindings":[{"var":".b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n0-n0":{"id":"n0-n0","name":"success","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.success\""},"n0-n1":{"id":"n0-n1","name":"failed","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.failed\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.bool_input_wf"},"interface":{"inputs":{"variables":{"b":{"type":{"simple":"BOOLEAN"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"b":{"scalar":{"primitive":{"boolean":false}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.failed\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.failed"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","failed"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.success\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.success"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","success"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-bool-input-wf"}},"spec":{"id":"::core.control_flow.conditions.bool_input_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"test","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"leftValue":{"var":".b"},"rightValue":{"primitive":{"boolean":true}}}},"then":"n0-n0"},"else":"n0-n1"},"inputBindings":[{"var":".b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"n0-n0":{"id":"n0-n0","name":"success","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.success\""},"n0-n1":{"id":"n0-n1","name":"failed","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.failed\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.bool_input_wf"},"interface":{"inputs":{"variables":{"b":{"type":{"simple":"BOOLEAN"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"b":{"scalar":{"primitive":{"boolean":false}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.failed\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.failed"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","failed"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.success\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.success"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","success"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/039_core.control_flow.conditions.nested_conditions_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/039_core.control_flow.conditions.nested_conditions_2_wf_crd.json index e40ae477ca..d75e2600ce 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/039_core.control_flow.conditions.nested_conditions_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/039_core.control_flow.conditions.nested_conditions_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-nested-conditions"}},"spec":{"id":"::core.control_flow.conditions.nested_conditions","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":10}}}}}},"then":"n0-n1"}],"else":"n0-n2"},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"inner_fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.5}}}},"then":"n0-n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.5}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.7}}}}}},"then":"n0-n0-n1"}],"elseFail":{"failed_node_id":"inner_fractions","message":"Only \u003c0.7 allowed"}},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0-n0":{"id":"n0-n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0-n1":{"id":"n0-n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n2":{"id":"n0-n2","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n0-n0":["start-node"],"n0-n0-n1":["start-node"],"n0-n1":["start-node"],"n0-n2":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.nested_conditions"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-nested-conditions"}},"spec":{"id":"::core.control_flow.conditions.nested_conditions","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}}}},"then":"n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":1}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":10}}}}}},"then":"n0-n1"}],"else":"n0-n2"},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0":{"id":"n0-n0","name":"inner_fractions","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.5}}}},"then":"n0-n0-n0"},"elseIf":[{"condition":{"conjunction":{"leftExpression":{"comparison":{"operator":"GT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.5}}}},"rightExpression":{"comparison":{"operator":"LT","leftValue":{"var":".my_input"},"rightValue":{"primitive":{"floatValue":0.7}}}}}},"then":"n0-n0-n1"}],"elseFail":{"failed_node_id":"inner_fractions","message":"Only \u003c0.7 allowed"}},"inputBindings":[{"var":".my_input","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0-n0":{"id":"n0-n0-n0","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n0-n1":{"id":"n0-n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n1":{"id":"n0-n1","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n0-n2":{"id":"n0-n2","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n0-n0":["start-node"],"n0-n0-n1":["start-node"],"n0-n1":["start-node"],"n0-n2":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.nested_conditions"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/042_core.control_flow.conditions.consume_outputs_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/042_core.control_flow.conditions.consume_outputs_2_wf_crd.json index 1bddedf27b..9f94863a61 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/042_core.control_flow.conditions.consume_outputs_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/042_core.control_flow.conditions.consume_outputs_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-consume-outputs"}},"spec":{"id":"::core.control_flow.conditions.consume_outputs","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"coin_toss","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"","inputBindings":[{"var":"seed","binding":{"promise":{"nodeId":"start-node","var":"seed"}}}]},"n1":{"id":"n1","name":"double_or_square","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"leftValue":{"var":"n0.o0"},"rightValue":{"primitive":{"boolean":true}}}},"then":"n1-n0"},"else":"n1-n1"},"inputBindings":[{"var":"n0.o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n1-n0":{"id":"n1-n0","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n1-n1":{"id":"n1-n1","name":"calc_sum","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.calc_sum\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n2":{"id":"n2","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n2"],"n0":["start-node"],"n1":["n0"],"n1-n0":["start-node"],"n1-n1":["start-node"],"n2":["n1"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.consume_outputs"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}},"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}},"seed":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.calc_sum\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.calc_sum"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","calc_sum"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.coin_toss"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"BOOLEAN"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","coin_toss"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-conditions-consume-outputs"}},"spec":{"id":"::core.control_flow.conditions.consume_outputs","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"coin_toss","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"","inputBindings":[{"var":"seed","binding":{"promise":{"nodeId":"start-node","var":"seed"}}}]},"n1":{"id":"n1","name":"double_or_square","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"leftValue":{"var":"n0.o0"},"rightValue":{"primitive":{"boolean":true}}}},"then":"n1-n0"},"else":"n1-n1"},"inputBindings":[{"var":"n0.o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n1-n0":{"id":"n1-n0","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.square\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n1-n1":{"id":"n1-n1","name":"calc_sum","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.calc_sum\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"n2":{"id":"n2","name":"double","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.conditions.double\"","inputBindings":[{"var":"n","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n2"],"n0":["start-node"],"n1":["n0"],"n1-n0":["start-node"],"n1-n1":["start-node"],"n2":["n1"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.conditions.consume_outputs"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"FLOAT"}},"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"floatValue":0}}},"seed":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.conditions.calc_sum\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.calc_sum"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"FLOAT"}},"b":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","calc_sum"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.coin_toss\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.coin_toss"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"seed":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"BOOLEAN"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","coin_toss"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.double\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.double"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","double"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.conditions.square\"":{"id":{"resourceType":"TASK","name":"core.control_flow.conditions.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"n":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.conditions","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/048_core.control_flow.dynamics.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/048_core.control_flow.dynamics.wf_2_wf_crd.json index 064f3e3b9c..3120082b6e 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/048_core.control_flow.dynamics.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/048_core.control_flow.dynamics.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-dynamics-wf"}},"spec":{"id":"::core.control_flow.dynamics.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"count_characters","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.dynamics.count_characters\"","inputBindings":[{"var":"s1","binding":{"promise":{"nodeId":"start-node","var":"s1"}}},{"var":"s2","binding":{"promise":{"nodeId":"start-node","var":"s2"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.dynamics.wf"},"interface":{"inputs":{"variables":{"s1":{"type":{"simple":"STRING"}},"s2":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"s1":{"scalar":{"primitive":{"stringValue":""}}},"s2":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.dynamics.count_characters\"":{"id":{"resourceType":"TASK","name":"core.control_flow.dynamics.count_characters"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"s1":{"type":{"simple":"STRING"}},"s2":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.dynamics","task-name","count_characters"],"resources":{},"env":[{"key":"_F_SS_C","value":"H4sIAAAAAAAC/8VRXW7DIAy+ysTzWtKtUuiuMk2RAZN6pZCBE6mrevcBXdXuBHsIsr8f85mcBR1hxMHE4GgUb09nYdHB7HloREMCHGtxY8Tzk3BfoSLj3qQ1Ren8iTGm8VqYGA+6fFXIUKcKExOulFa4AbXdmR1YZ0CpbruDrt/qTWfBvvTK9V3X9+JSjO36XLzv/xHgo0aYUvxEw8UeZu8LYOMRKNz7BVOm+ABgWO7NdOJ9DAMFxjQlLGcNIuPEcilCqSnIq+a1bVSTH4iHhRLP4ItkSDHyX5MnfTOtlczEuJrAHOpbyduENg0yDxkTgadv4JKydMwUxtz+KQbQHm2pHfiMdTnMhb5KLaWHrSlzIj03xkfTJL/05fIDQ+X0kUICAAA="}],"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-dynamics-wf"}},"spec":{"id":"::core.control_flow.dynamics.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"count_characters","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.dynamics.count_characters\"","inputBindings":[{"var":"s1","binding":{"promise":{"nodeId":"start-node","var":"s1"}}},{"var":"s2","binding":{"promise":{"nodeId":"start-node","var":"s2"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.dynamics.wf"},"interface":{"inputs":{"variables":{"s1":{"type":{"simple":"STRING"}},"s2":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"s1":{"scalar":{"primitive":{"stringValue":""}}},"s2":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.dynamics.count_characters\"":{"id":{"resourceType":"TASK","name":"core.control_flow.dynamics.count_characters"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"s1":{"type":{"simple":"STRING"}},"s2":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.dynamics","task-name","count_characters"],"resources":{},"env":[{"key":"_F_SS_C","value":"H4sIAAAAAAAC/8VRXW7DIAy+ysTzWtKtUuiuMk2RAZN6pZCBE6mrevcBXdXuBHsIsr8f85mcBR1hxMHE4GgUb09nYdHB7HloREMCHGtxY8Tzk3BfoSLj3qQ1Ren8iTGm8VqYGA+6fFXIUKcKExOulFa4AbXdmR1YZ0CpbruDrt/qTWfBvvTK9V3X9+JSjO36XLzv/xHgo0aYUvxEw8UeZu8LYOMRKNz7BVOm+ABgWO7NdOJ9DAMFxjQlLGcNIuPEcilCqSnIq+a1bVSTH4iHhRLP4ItkSDHyX5MnfTOtlczEuJrAHOpbyduENg0yDxkTgadv4JKydMwUxtz+KQbQHm2pHfiMdTnMhb5KLaWHrSlzIj03xkfTJL/05fIDQ+X0kUICAAA="}],"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/053_core.control_flow.map_task.my_map_workflow_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/053_core.control_flow.map_task.my_map_workflow_2_wf_crd.json index 8ebe2e607c..edbb53b110 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/053_core.control_flow.map_task.my_map_workflow_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/053_core.control_flow.map_task.my_map_workflow_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-map-task-my-map-workflow"}},"spec":{"id":"::core.control_flow.map_task.my_map_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"mapper_a_mappable_task_0","resources":{"limits":{"memory":"500Mi"},"requests":{"memory":"300Mi"}},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.map_task.mapper_a_mappable_task_0\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}],"retry":{"minAttempts":2}},"n1":{"id":"n1","name":"coalesce","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.map_task.coalesce\"","inputBindings":[{"var":"b","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.map_task.my_map_workflow"},"interface":{"inputs":{"variables":{"a":{"type":{"collectionType":{"simple":"INTEGER"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"a":{"collection":{"literals":[{"scalar":{"primitive":{"integer":"0"}}}]}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.map_task.coalesce\"":{"id":{"resourceType":"TASK","name":"core.control_flow.map_task.coalesce"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"b":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.map_task","task-name","coalesce"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.map_task.mapper_a_mappable_task_0\"":{"id":{"resourceType":"TASK","name":"core.control_flow.map_task.mapper_a_mappable_task_0"},"type":"container_array","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"collectionType":{"simple":"INTEGER"}}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"STRING"}}}}}},"custom":{"minSuccessRatio":1},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-map-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.map_task","task-name","a_mappable_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"taskTypeVersion":1}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-map-task-my-map-workflow"}},"spec":{"id":"::core.control_flow.map_task.my_map_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"mapper_a_mappable_task_0","resources":{"limits":{"memory":"500Mi"},"requests":{"memory":"300Mi"}},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.map_task.mapper_a_mappable_task_0\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}],"retry":{"minAttempts":2}},"n1":{"id":"n1","name":"coalesce","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.map_task.coalesce\"","inputBindings":[{"var":"b","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.map_task.my_map_workflow"},"interface":{"inputs":{"variables":{"a":{"type":{"collectionType":{"simple":"INTEGER"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"a":{"collection":{"literals":[{"scalar":{"primitive":{"integer":"0"}}}]}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.map_task.coalesce\"":{"id":{"resourceType":"TASK","name":"core.control_flow.map_task.coalesce"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"b":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.map_task","task-name","coalesce"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.map_task.mapper_a_mappable_task_0\"":{"id":{"resourceType":"TASK","name":"core.control_flow.map_task.mapper_a_mappable_task_0"},"type":"container_array","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"collectionType":{"simple":"INTEGER"}}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"STRING"}}}}}},"custom":{"minSuccessRatio":1},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-map-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.map_task","task-name","a_mappable_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]},"taskTypeVersion":1}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/059_core.control_flow.merge_sort.merge_sort_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/059_core.control_flow.merge_sort.merge_sort_2_wf_crd.json index a806ef9123..6f26fbc8bf 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/059_core.control_flow.merge_sort.merge_sort_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/059_core.control_flow.merge_sort.merge_sort_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-merge-sort-merge-sort"}},"spec":{"id":"::core.control_flow.merge_sort.merge_sort","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"terminal_case","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"operator":"LTE","leftValue":{"var":".numbers_count"},"rightValue":{"var":".run_local_at_count"}}},"then":"n0-n0"},"else":"n0-n1"},"inputBindings":[{"var":".numbers_count","binding":{"promise":{"nodeId":"start-node","var":"numbers_count"}}},{"var":".run_local_at_count","binding":{"promise":{"nodeId":"start-node","var":"run_local_at_count"}}}]},"n0-n0":{"id":"n0-n0","name":"sort_locally","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.merge_sort.sort_locally\"","inputBindings":[{"var":"numbers","binding":{"promise":{"nodeId":"start-node","var":"numbers"}}}]},"n0-n1":{"id":"n0-n1","name":"merge_sort_remotely","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.merge_sort.merge_sort_remotely\"","inputBindings":[{"var":"numbers","binding":{"promise":{"nodeId":"start-node","var":"numbers"}}},{"var":"run_local_at_count","binding":{"promise":{"nodeId":"start-node","var":"run_local_at_count"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.merge_sort.merge_sort"},"interface":{"inputs":{"variables":{"numbers":{"type":{"collectionType":{"simple":"INTEGER"}}},"numbers_count":{"type":{"simple":"INTEGER"}},"run_local_at_count":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"INTEGER"}}}}}}},"inputs":{"literals":{"numbers":{"collection":{"literals":[{"scalar":{"primitive":{"integer":"0"}}}]}},"numbers_count":{"scalar":{"primitive":{"integer":"0"}}},"run_local_at_count":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.merge_sort.merge_sort_remotely\"":{"id":{"resourceType":"TASK","name":"core.control_flow.merge_sort.merge_sort_remotely"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"numbers":{"type":{"collectionType":{"simple":"INTEGER"}}},"run_local_at_count":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"INTEGER"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.merge_sort","task-name","merge_sort_remotely"],"resources":{},"env":[{"key":"_F_SS_C","value":"H4sIAAAAAAAC/8VRXW7DIAy+ysTzWtKtUuiuMk2RAZN6pZCBE6mrevcBXdXuBHsIsr8f85mcBR1hxMHE4GgUb09nYdHB7HloREMCHGtxY8Tzk3BfoSLj3qQ1Ren8iTGm8VqYGA+6fFXIUKcKExOulFa4AbXdmR1YZ0CpbruDrt/qTWfBvvTK9V3X9+JSjO36XLzv/xHgo0aYUvxEw8UeZu8LYOMRKNz7BVOm+ABgWO7NdOJ9DAMFxjQlLGcNIuPEcilCqSnIq+a1bVSTH4iHhRLP4ItkSDHyX5MnfTOtlczEuJrAHOpbyduENg0yDxkTgadv4JKydMwUxtz+KQbQHm2pHfiMdTnMhb5KLaWHrSlzIj03xkfTJL/05fIDQ+X0kUICAAA="}],"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.merge_sort.sort_locally\"":{"id":{"resourceType":"TASK","name":"core.control_flow.merge_sort.sort_locally"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"numbers":{"type":{"collectionType":{"simple":"INTEGER"}}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"INTEGER"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.merge_sort","task-name","sort_locally"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-merge-sort-merge-sort"}},"spec":{"id":"::core.control_flow.merge_sort.merge_sort","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"terminal_case","resources":{},"kind":"branch","branch":{"if":{"condition":{"comparison":{"operator":"LTE","leftValue":{"var":".numbers_count"},"rightValue":{"var":".run_local_at_count"}}},"then":"n0-n0"},"else":"n0-n1"},"inputBindings":[{"var":".numbers_count","binding":{"promise":{"nodeId":"start-node","var":"numbers_count"}}},{"var":".run_local_at_count","binding":{"promise":{"nodeId":"start-node","var":"run_local_at_count"}}}]},"n0-n0":{"id":"n0-n0","name":"sort_locally","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.merge_sort.sort_locally\"","inputBindings":[{"var":"numbers","binding":{"promise":{"nodeId":"start-node","var":"numbers"}}}]},"n0-n1":{"id":"n0-n1","name":"merge_sort_remotely","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.merge_sort.merge_sort_remotely\"","inputBindings":[{"var":"numbers","binding":{"promise":{"nodeId":"start-node","var":"numbers"}}},{"var":"run_local_at_count","binding":{"promise":{"nodeId":"start-node","var":"run_local_at_count"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"],"n0-n0":["start-node"],"n0-n1":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.merge_sort.merge_sort"},"interface":{"inputs":{"variables":{"numbers":{"type":{"collectionType":{"simple":"INTEGER"}}},"numbers_count":{"type":{"simple":"INTEGER"}},"run_local_at_count":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"INTEGER"}}}}}}},"inputs":{"literals":{"numbers":{"collection":{"literals":[{"scalar":{"primitive":{"integer":"0"}}}]}},"numbers_count":{"scalar":{"primitive":{"integer":"0"}}},"run_local_at_count":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.merge_sort.merge_sort_remotely\"":{"id":{"resourceType":"TASK","name":"core.control_flow.merge_sort.merge_sort_remotely"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"numbers":{"type":{"collectionType":{"simple":"INTEGER"}}},"run_local_at_count":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"INTEGER"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.merge_sort","task-name","merge_sort_remotely"],"resources":{},"env":[{"key":"_F_SS_C","value":"H4sIAAAAAAAC/8VRXW7DIAy+ysTzWtKtUuiuMk2RAZN6pZCBE6mrevcBXdXuBHsIsr8f85mcBR1hxMHE4GgUb09nYdHB7HloREMCHGtxY8Tzk3BfoSLj3qQ1Ren8iTGm8VqYGA+6fFXIUKcKExOulFa4AbXdmR1YZ0CpbruDrt/qTWfBvvTK9V3X9+JSjO36XLzv/xHgo0aYUvxEw8UeZu8LYOMRKNz7BVOm+ABgWO7NdOJ9DAMFxjQlLGcNIuPEcilCqSnIq+a1bVSTH4iHhRLP4ItkSDHyX5MnfTOtlczEuJrAHOpbyduENg0yDxkTgadv4JKydMwUxtz+KQbQHm2pHfiMdTnMhb5KLaWHrSlzIj03xkfTJL/05fIDQ+X0kUICAAA="}],"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.control_flow.merge_sort.sort_locally\"":{"id":{"resourceType":"TASK","name":"core.control_flow.merge_sort.sort_locally"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"numbers":{"type":{"collectionType":{"simple":"INTEGER"}}}}},"outputs":{"variables":{"o0":{"type":{"collectionType":{"simple":"INTEGER"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.merge_sort","task-name","sort_locally"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/062_core.control_flow.subworkflows.my_subwf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/062_core.control_flow.subworkflows.my_subwf_2_wf_crd.json index 863a62b718..2e0e8073d3 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/062_core.control_flow.subworkflows.my_subwf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/062_core.control_flow.subworkflows.my_subwf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-subworkflows-my-subwf"}},"spec":{"id":"::core.control_flow.subworkflows.my_subwf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"c"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"c"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.subworkflows.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.subworkflows.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"n0","var":"t1_int_output"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"c"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"c"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.subworkflows.my_subwf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.subworkflows.t1\"":{"id":{"resourceType":"TASK","name":"core.control_flow.subworkflows.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"c":{"type":{"simple":"STRING"}},"t1_int_output":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.subworkflows","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-subworkflows-my-subwf"}},"spec":{"id":"::core.control_flow.subworkflows.my_subwf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"c"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"c"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.subworkflows.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.subworkflows.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"n0","var":"t1_int_output"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"c"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"c"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.subworkflows.my_subwf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}},"o1":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.subworkflows.t1\"":{"id":{"resourceType":"TASK","name":"core.control_flow.subworkflows.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"c":{"type":{"simple":"STRING"}},"t1_int_output":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.subworkflows","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/069_core.control_flow.subworkflows.ext_workflow_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/069_core.control_flow.subworkflows.ext_workflow_2_wf_crd.json index d6ebc6f43a..e4a21e3c44 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/069_core.control_flow.subworkflows.ext_workflow_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/069_core.control_flow.subworkflows.ext_workflow_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-subworkflows-ext-workflow"}},"spec":{"id":"::core.control_flow.subworkflows.ext_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"count_freq_words","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.subworkflows.count_freq_words\"","inputBindings":[{"var":"input_string1","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.subworkflows.ext_workflow"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.subworkflows.count_freq_words\"":{"id":{"resourceType":"TASK","name":"core.control_flow.subworkflows.count_freq_words"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"input_string1":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.subworkflows","task-name","count_freq_words"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-control-flow-subworkflows-ext-workflow"}},"spec":{"id":"::core.control_flow.subworkflows.ext_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"count_freq_words","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.control_flow.subworkflows.count_freq_words\"","inputBindings":[{"var":"input_string1","binding":{"promise":{"nodeId":"start-node","var":"my_input"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.control_flow.subworkflows.ext_workflow"},"interface":{"inputs":{"variables":{"my_input":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}}},"inputs":{"literals":{"my_input":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.control_flow.subworkflows.count_freq_words\"":{"id":{"resourceType":"TASK","name":"core.control_flow.subworkflows.count_freq_words"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"input_string1":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.control_flow.subworkflows","task-name","count_freq_words"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/077_core.extend_flyte.custom_task_plugin.my_workflow_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/077_core.extend_flyte.custom_task_plugin.my_workflow_2_wf_crd.json index 001a4dd641..eaaf68c03c 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/077_core.extend_flyte.custom_task_plugin.my_workflow_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/077_core.extend_flyte.custom_task_plugin.my_workflow_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-extend-flyte-custom-task-plugin-my-workflow"}},"spec":{"id":"::core.extend_flyte.custom_task_plugin.my_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"my-objectstore-sensor","resources":{},"kind":"task","task":"resource_type:TASK name:\"my-objectstore-sensor\"","inputBindings":[{"var":"path","binding":{"promise":{"nodeId":"start-node","var":"path"}}}],"retry":{"minAttempts":11},"executionDeadline":"20m0s","activeDeadline":"40m0s"},"n1":{"id":"n1","name":"print_file","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.extend_flyte.custom_task_plugin.print_file\"","inputBindings":[{"var":"path","binding":{"promise":{"nodeId":"n0","var":"path"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.extend_flyte.custom_task_plugin.my_workflow"},"interface":{"inputs":{"variables":{"path":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"path":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.extend_flyte.custom_task_plugin.print_file\"":{"id":{"resourceType":"TASK","name":"core.extend_flyte.custom_task_plugin.print_file"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"path":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.extend_flyte.custom_task_plugin","task-name","print_file"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"my-objectstore-sensor\"":{"id":{"resourceType":"TASK","name":"my-objectstore-sensor"},"type":"object-store-sensor","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"timeout":"1200s","retries":{"retries":10}},"interface":{"inputs":{"variables":{"path":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"path":{"type":{"simple":"STRING"}}}}}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-extend-flyte-custom-task-plugin-my-workflow"}},"spec":{"id":"::core.extend_flyte.custom_task_plugin.my_workflow","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"my-objectstore-sensor","resources":{},"kind":"task","task":"resource_type:TASK name:\"my-objectstore-sensor\"","inputBindings":[{"var":"path","binding":{"promise":{"nodeId":"start-node","var":"path"}}}],"retry":{"minAttempts":11},"executionDeadline":"20m0s","activeDeadline":"40m0s"},"n1":{"id":"n1","name":"print_file","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.extend_flyte.custom_task_plugin.print_file\"","inputBindings":[{"var":"path","binding":{"promise":{"nodeId":"n0","var":"path"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.extend_flyte.custom_task_plugin.my_workflow"},"interface":{"inputs":{"variables":{"path":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"path":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.extend_flyte.custom_task_plugin.print_file\"":{"id":{"resourceType":"TASK","name":"core.extend_flyte.custom_task_plugin.print_file"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"path":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.extend_flyte.custom_task_plugin","task-name","print_file"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"my-objectstore-sensor\"":{"id":{"resourceType":"TASK","name":"my-objectstore-sensor"},"type":"object-store-sensor","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"timeout":"1200s","retries":{"retries":10}},"interface":{"inputs":{"variables":{"path":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"path":{"type":{"simple":"STRING"}}}}}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/081_core.extend_flyte.custom_types.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/081_core.extend_flyte.custom_types.wf_2_wf_crd.json index 00cb79828a..43ed50b46a 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/081_core.extend_flyte.custom_types.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/081_core.extend_flyte.custom_types.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-extend-flyte-custom-types-wf"}},"spec":{"id":"::core.extend_flyte.custom_types.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"generate","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.extend_flyte.custom_types.generate\""},"n1":{"id":"n1","name":"consume","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.extend_flyte.custom_types.consume\"","inputBindings":[{"var":"d","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.extend_flyte.custom_types.wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.extend_flyte.custom_types.consume\"":{"id":{"resourceType":"TASK","name":"core.extend_flyte.custom_types.consume"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"d":{"type":{"blob":{"format":"binary","dimensionality":"MULTIPART"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.extend_flyte.custom_types","task-name","consume"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.extend_flyte.custom_types.generate\"":{"id":{"resourceType":"TASK","name":"core.extend_flyte.custom_types.generate"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"blob":{"format":"binary","dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.extend_flyte.custom_types","task-name","generate"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-extend-flyte-custom-types-wf"}},"spec":{"id":"::core.extend_flyte.custom_types.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"generate","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.extend_flyte.custom_types.generate\""},"n1":{"id":"n1","name":"consume","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.extend_flyte.custom_types.consume\"","inputBindings":[{"var":"d","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.extend_flyte.custom_types.wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.extend_flyte.custom_types.consume\"":{"id":{"resourceType":"TASK","name":"core.extend_flyte.custom_types.consume"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"d":{"type":{"blob":{"format":"binary","dimensionality":"MULTIPART"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.extend_flyte.custom_types","task-name","consume"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.extend_flyte.custom_types.generate\"":{"id":{"resourceType":"TASK","name":"core.extend_flyte.custom_types.generate"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"blob":{"format":"binary","dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.extend_flyte.custom_types","task-name","generate"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/085_core.flyte_basics.basic_workflow.my_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/085_core.flyte_basics.basic_workflow.my_wf_2_wf_crd.json index 230c3eab3e..406a2a464c 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/085_core.flyte_basics.basic_workflow.my_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/085_core.flyte_basics.basic_workflow.my_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-basic-workflow-my-wf"}},"spec":{"id":"::core.flyte_basics.basic_workflow.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"t1_int_output"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t2\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"n0","var":"c"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["n0","start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"t1_int_output"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.basic_workflow.my_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}},"b":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}},"o1":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}},"b":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.basic_workflow.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"c":{"type":{"simple":"STRING"}},"t1_int_output":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.basic_workflow","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.basic_workflow.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"STRING"}},"b":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.basic_workflow","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-basic-workflow-my-wf"}},"spec":{"id":"::core.flyte_basics.basic_workflow.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"t1_int_output"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t2\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"n0","var":"c"}}},{"var":"b","binding":{"promise":{"nodeId":"start-node","var":"b"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["n0","start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"t1_int_output"}}},{"var":"o1","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.basic_workflow.my_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}},"b":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}},"o1":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}},"b":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.basic_workflow.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"c":{"type":{"simple":"STRING"}},"t1_int_output":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.basic_workflow","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.basic_workflow.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.basic_workflow.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"STRING"}},"b":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.basic_workflow","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/089_core.flyte_basics.decorating_tasks.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/089_core.flyte_basics.decorating_tasks.wf_2_wf_crd.json index 59877193d6..5c2452ce4f 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/089_core.flyte_basics.decorating_tasks.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/089_core.flyte_basics.decorating_tasks.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-decorating-tasks-wf"}},"spec":{"id":"::core.flyte_basics.decorating_tasks.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t1\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"x"}}}]},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t2\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.decorating_tasks.wf"},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"x":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_tasks.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_tasks","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_tasks.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_tasks","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-decorating-tasks-wf"}},"spec":{"id":"::core.flyte_basics.decorating_tasks.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t1\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"x"}}}]},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t2\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.decorating_tasks.wf"},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"x":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_tasks.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_tasks","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_tasks.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_tasks.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_tasks","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/095_core.flyte_basics.decorating_workflows.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/095_core.flyte_basics.decorating_workflows.wf_2_wf_crd.json index eb76ca6a7b..63196d86fc 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/095_core.flyte_basics.decorating_workflows.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/095_core.flyte_basics.decorating_workflows.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-decorating-workflows-wf"}},"spec":{"id":"::core.flyte_basics.decorating_workflows.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"setup","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.setup\""},"n1":{"id":"n1","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t1\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"x"}}}]},"n2":{"id":"n2","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t2\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n3":{"id":"n3","name":"teardown","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.teardown\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node","n3"],"n3":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node","n3"],"n3":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n2","n3"],"n0":["start-node"],"n1":["n0","start-node"],"n2":["n1"],"n3":["n2"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.decorating_workflows.wf"},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"x":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.setup\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.setup"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","setup"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.teardown\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.teardown"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","teardown"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-decorating-workflows-wf"}},"spec":{"id":"::core.flyte_basics.decorating_workflows.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"setup","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.setup\""},"n1":{"id":"n1","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t1\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"x"}}}]},"n2":{"id":"n2","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t2\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n3":{"id":"n3","name":"teardown","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.teardown\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node","n3"],"n3":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node","n3"],"n3":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n2","n3"],"n0":["start-node"],"n1":["n0","start-node"],"n2":["n1"],"n3":["n2"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.decorating_workflows.wf"},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}}},"inputs":{"literals":{"x":{"scalar":{"primitive":{"floatValue":0}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.setup\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.setup"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","setup"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"FLOAT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"FLOAT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.decorating_workflows.teardown\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.decorating_workflows.teardown"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.decorating_workflows","task-name","teardown"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/098_core.flyte_basics.documented_workflow.sphinx_docstring_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/098_core.flyte_basics.documented_workflow.sphinx_docstring_2_wf_crd.json index 40ffb6b2e4..1b9dabd6af 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/098_core.flyte_basics.documented_workflow.sphinx_docstring_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/098_core.flyte_basics.documented_workflow.sphinx_docstring_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-documented-workflow-sphinx-docstring"}},"spec":{"id":"::core.flyte_basics.documented_workflow.sphinx_docstring","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"add_data","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"","inputBindings":[{"var":"data","binding":{"promise":{"nodeId":"start-node","var":"data"}}},{"var":"df","binding":{"promise":{"nodeId":"start-node","var":"df"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.documented_workflow.sphinx_docstring"},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"data":{"scalar":{"generic":{}}},"df":{"scalar":{"structuredDataset":{"metadata":{"structuredDatasetType":{"format":"parquet"}}}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.documented_workflow.add_data"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.documented_workflow","task-name","add_data"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-documented-workflow-sphinx-docstring"}},"spec":{"id":"::core.flyte_basics.documented_workflow.sphinx_docstring","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"add_data","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"","inputBindings":[{"var":"data","binding":{"promise":{"nodeId":"start-node","var":"data"}}},{"var":"df","binding":{"promise":{"nodeId":"start-node","var":"df"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.documented_workflow.sphinx_docstring"},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"data":{"scalar":{"generic":{}}},"df":{"scalar":{"structuredDataset":{"metadata":{"structuredDatasetType":{"format":"parquet"}}}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.documented_workflow.add_data"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.documented_workflow","task-name","add_data"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/100_core.flyte_basics.documented_workflow.numpy_docstring_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/100_core.flyte_basics.documented_workflow.numpy_docstring_2_wf_crd.json index 8c06eff7dc..ffa46c719b 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/100_core.flyte_basics.documented_workflow.numpy_docstring_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/100_core.flyte_basics.documented_workflow.numpy_docstring_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-documented-workflow-numpy-docstring"}},"spec":{"id":"::core.flyte_basics.documented_workflow.numpy_docstring","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"add_data","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"","inputBindings":[{"var":"data","binding":{"promise":{"nodeId":"start-node","var":"data"}}},{"var":"df","binding":{"promise":{"nodeId":"start-node","var":"df"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.documented_workflow.numpy_docstring"},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"data":{"scalar":{"generic":{}}},"df":{"scalar":{"structuredDataset":{"metadata":{"structuredDatasetType":{"format":"parquet"}}}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.documented_workflow.add_data"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.documented_workflow","task-name","add_data"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-documented-workflow-numpy-docstring"}},"spec":{"id":"::core.flyte_basics.documented_workflow.numpy_docstring","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"add_data","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"","inputBindings":[{"var":"data","binding":{"promise":{"nodeId":"start-node","var":"data"}}},{"var":"df","binding":{"promise":{"nodeId":"start-node","var":"df"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.documented_workflow.numpy_docstring"},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"data":{"scalar":{"generic":{}}},"df":{"scalar":{"structuredDataset":{"metadata":{"structuredDatasetType":{"format":"parquet"}}}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.documented_workflow.add_data"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.documented_workflow","task-name","add_data"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/102_core.flyte_basics.documented_workflow.google_docstring_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/102_core.flyte_basics.documented_workflow.google_docstring_2_wf_crd.json index d82370907a..4c1432f6f9 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/102_core.flyte_basics.documented_workflow.google_docstring_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/102_core.flyte_basics.documented_workflow.google_docstring_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-documented-workflow-google-docstring"}},"spec":{"id":"::core.flyte_basics.documented_workflow.google_docstring","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"add_data","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"","inputBindings":[{"var":"data","binding":{"promise":{"nodeId":"start-node","var":"data"}}},{"var":"df","binding":{"promise":{"nodeId":"start-node","var":"df"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.documented_workflow.google_docstring"},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"data":{"scalar":{"generic":{}}},"df":{"scalar":{"structuredDataset":{"metadata":{"structuredDatasetType":{"format":"parquet"}}}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.documented_workflow.add_data"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.documented_workflow","task-name","add_data"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-documented-workflow-google-docstring"}},"spec":{"id":"::core.flyte_basics.documented_workflow.google_docstring","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"add_data","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"","inputBindings":[{"var":"data","binding":{"promise":{"nodeId":"start-node","var":"data"}}},{"var":"df","binding":{"promise":{"nodeId":"start-node","var":"df"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.documented_workflow.google_docstring"},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"data":{"scalar":{"generic":{}}},"df":{"scalar":{"structuredDataset":{"metadata":{"structuredDatasetType":{"format":"parquet"}}}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.documented_workflow.add_data\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.documented_workflow.add_data"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"data":{"type":{"simple":"STRUCT"}},"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.documented_workflow","task-name","add_data"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/105_core.flyte_basics.files.normalize_csv_file_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/105_core.flyte_basics.files.normalize_csv_file_2_wf_crd.json index a974f37eab..252f783ca1 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/105_core.flyte_basics.files.normalize_csv_file_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/105_core.flyte_basics.files.normalize_csv_file_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-files-normalize-csv-file"}},"spec":{"id":"::core.flyte_basics.files.normalize_csv_file","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"normalize_columns","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.files.normalize_columns\"","inputBindings":[{"var":"column_names","binding":{"promise":{"nodeId":"start-node","var":"column_names"}}},{"var":"columns_to_normalize","binding":{"promise":{"nodeId":"start-node","var":"columns_to_normalize"}}},{"var":"csv_url","binding":{"promise":{"nodeId":"start-node","var":"csv_url"}}},{"var":"output_location","binding":{"promise":{"nodeId":"start-node","var":"output_location"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.files.normalize_csv_file"},"interface":{"inputs":{"variables":{"column_names":{"type":{"collectionType":{"simple":"STRING"}}},"columns_to_normalize":{"type":{"collectionType":{"simple":"STRING"}}},"csv_url":{"type":{"blob":{}}},"output_location":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{}}}}}}},"inputs":{"literals":{"column_names":{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}},"columns_to_normalize":{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}},"csv_url":{"scalar":{"blob":{"metadata":{"type":{}},"uri":"/tmp/somepath"}}},"output_location":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.files.normalize_columns\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.files.normalize_columns"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"column_names":{"type":{"collectionType":{"simple":"STRING"}}},"columns_to_normalize":{"type":{"collectionType":{"simple":"STRING"}}},"csv_url":{"type":{"blob":{}}},"output_location":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.files","task-name","normalize_columns"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-files-normalize-csv-file"}},"spec":{"id":"::core.flyte_basics.files.normalize_csv_file","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"normalize_columns","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.files.normalize_columns\"","inputBindings":[{"var":"column_names","binding":{"promise":{"nodeId":"start-node","var":"column_names"}}},{"var":"columns_to_normalize","binding":{"promise":{"nodeId":"start-node","var":"columns_to_normalize"}}},{"var":"csv_url","binding":{"promise":{"nodeId":"start-node","var":"csv_url"}}},{"var":"output_location","binding":{"promise":{"nodeId":"start-node","var":"output_location"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.files.normalize_csv_file"},"interface":{"inputs":{"variables":{"column_names":{"type":{"collectionType":{"simple":"STRING"}}},"columns_to_normalize":{"type":{"collectionType":{"simple":"STRING"}}},"csv_url":{"type":{"blob":{}}},"output_location":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{}}}}}}},"inputs":{"literals":{"column_names":{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}},"columns_to_normalize":{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}},"csv_url":{"scalar":{"blob":{"metadata":{"type":{}},"uri":"/tmp/somepath"}}},"output_location":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.files.normalize_columns\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.files.normalize_columns"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"column_names":{"type":{"collectionType":{"simple":"STRING"}}},"columns_to_normalize":{"type":{"collectionType":{"simple":"STRING"}}},"csv_url":{"type":{"blob":{}}},"output_location":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.files","task-name","normalize_columns"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/109_core.flyte_basics.folders.download_and_normalize_csv_files_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/109_core.flyte_basics.folders.download_and_normalize_csv_files_2_wf_crd.json index e7cb29409d..3329624922 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/109_core.flyte_basics.folders.download_and_normalize_csv_files_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/109_core.flyte_basics.folders.download_and_normalize_csv_files_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-folders-download-and-normalize-csv-files"}},"spec":{"id":"::core.flyte_basics.folders.download_and_normalize_csv_files","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"download_files","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.folders.download_files\"","inputBindings":[{"var":"csv_urls","binding":{"promise":{"nodeId":"start-node","var":"csv_urls"}}}]},"n1":{"id":"n1","name":"normalize_all_files","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.folders.normalize_all_files\"","inputBindings":[{"var":"columns_metadata","binding":{"promise":{"nodeId":"start-node","var":"columns_metadata"}}},{"var":"columns_to_normalize_metadata","binding":{"promise":{"nodeId":"start-node","var":"columns_to_normalize_metadata"}}},{"var":"csv_files_dir","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0","start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.folders.download_and_normalize_csv_files"},"interface":{"inputs":{"variables":{"columns_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"columns_to_normalize_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"csv_urls":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}}},"inputs":{"literals":{"columns_metadata":{"collection":{"literals":[{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}}]}},"columns_to_normalize_metadata":{"collection":{"literals":[{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}}]}},"csv_urls":{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.folders.download_files\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.folders.download_files"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"csv_urls":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.folders","task-name","download_files"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.folders.normalize_all_files\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.folders.normalize_all_files"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"columns_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"columns_to_normalize_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"csv_files_dir":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.folders","task-name","normalize_all_files"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-folders-download-and-normalize-csv-files"}},"spec":{"id":"::core.flyte_basics.folders.download_and_normalize_csv_files","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"download_files","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.folders.download_files\"","inputBindings":[{"var":"csv_urls","binding":{"promise":{"nodeId":"start-node","var":"csv_urls"}}}]},"n1":{"id":"n1","name":"normalize_all_files","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.folders.normalize_all_files\"","inputBindings":[{"var":"columns_metadata","binding":{"promise":{"nodeId":"start-node","var":"columns_metadata"}}},{"var":"columns_to_normalize_metadata","binding":{"promise":{"nodeId":"start-node","var":"columns_to_normalize_metadata"}}},{"var":"csv_files_dir","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0","start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.folders.download_and_normalize_csv_files"},"interface":{"inputs":{"variables":{"columns_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"columns_to_normalize_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"csv_urls":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}}},"inputs":{"literals":{"columns_metadata":{"collection":{"literals":[{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}}]}},"columns_to_normalize_metadata":{"collection":{"literals":[{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}}]}},"csv_urls":{"collection":{"literals":[{"scalar":{"primitive":{"stringValue":""}}}]}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.folders.download_files\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.folders.download_files"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"csv_urls":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.folders","task-name","download_files"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.folders.normalize_all_files\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.folders.normalize_all_files"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"columns_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"columns_to_normalize_metadata":{"type":{"collectionType":{"collectionType":{"simple":"STRING"}}}},"csv_files_dir":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.folders","task-name","normalize_all_files"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/112_core.flyte_basics.hello_world.my_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/112_core.flyte_basics.hello_world.my_wf_2_wf_crd.json index 6f76c63baa..43bc526e24 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/112_core.flyte_basics.hello_world.my_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/112_core.flyte_basics.hello_world.my_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-hello-world-my-wf"}},"spec":{"id":"::core.flyte_basics.hello_world.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"say_hello","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.hello_world.say_hello\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.hello_world.my_wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.hello_world.say_hello\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.hello_world.say_hello"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.hello_world","task-name","say_hello"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-hello-world-my-wf"}},"spec":{"id":"::core.flyte_basics.hello_world.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"say_hello","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.hello_world.say_hello\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.hello_world.my_wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.hello_world.say_hello\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.hello_world.say_hello"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.hello_world","task-name","say_hello"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/117_my.imperative.workflow.example_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/117_my.imperative.workflow.example_2_wf_crd.json index eddb9423b2..0f01fc848e 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/117_my.imperative.workflow.example_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/117_my.imperative.workflow.example_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"my-imperative-workflow-example"}},"spec":{"id":"::my.imperative.workflow.example","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"output_from_t1","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"output_list","binding":{"collection":{"bindings":[{"promise":{"nodeId":"n0","var":"o0"}},{"promise":{"nodeId":"n2","var":"o0"}}]}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"in1"}}}]},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t2\""},"n2":{"id":"n2","name":"t3","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t3\"","inputBindings":[{"var":"a","binding":{"collection":{"bindings":[{"promise":{"nodeId":"start-node","var":"in1"}},{"promise":{"nodeId":"start-node","var":"in2"}}]}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"edges":{"downstream":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"upstream":{"end-node":["n0","n1","n2"],"n0":["start-node"],"n1":["start-node"],"n2":["start-node"]}},"outputBindings":[{"var":"output_from_t1","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"output_list","binding":{"collection":{"bindings":[{"promise":{"nodeId":"n0","var":"o0"}},{"promise":{"nodeId":"n2","var":"o0"}}]}}}],"identifier":{"resourceType":"WORKFLOW","name":"my.imperative.workflow.example"},"interface":{"inputs":{"variables":{"in1":{"type":{"simple":"STRING"}},"in2":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"output_from_t1":{"type":{"simple":"STRING"}},"output_list":{"type":{"collectionType":{"simple":"STRING"}}}}}}},"inputs":{"literals":{"in1":{"scalar":{"primitive":{"stringValue":""}}},"in2":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.imperative_wf_style.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.imperative_wf_style","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.imperative_wf_style.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.imperative_wf_style","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t3\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.imperative_wf_style.t3"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.imperative_wf_style","task-name","t3"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"my-imperative-workflow-example"}},"spec":{"id":"::my.imperative.workflow.example","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"output_from_t1","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"output_list","binding":{"collection":{"bindings":[{"promise":{"nodeId":"n0","var":"o0"}},{"promise":{"nodeId":"n2","var":"o0"}}]}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t1\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"in1"}}}]},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t2\""},"n2":{"id":"n2","name":"t3","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t3\"","inputBindings":[{"var":"a","binding":{"collection":{"bindings":[{"promise":{"nodeId":"start-node","var":"in1"}},{"promise":{"nodeId":"start-node","var":"in2"}}]}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"edges":{"downstream":{"n0":["end-node"],"n1":["end-node"],"n2":["end-node"],"start-node":["n0","n1","n2"]},"upstream":{"end-node":["n0","n1","n2"],"n0":["start-node"],"n1":["start-node"],"n2":["start-node"]}},"outputBindings":[{"var":"output_from_t1","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"output_list","binding":{"collection":{"bindings":[{"promise":{"nodeId":"n0","var":"o0"}},{"promise":{"nodeId":"n2","var":"o0"}}]}}}],"identifier":{"resourceType":"WORKFLOW","name":"my.imperative.workflow.example"},"interface":{"inputs":{"variables":{"in1":{"type":{"simple":"STRING"}},"in2":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"output_from_t1":{"type":{"simple":"STRING"}},"output_list":{"type":{"collectionType":{"simple":"STRING"}}}}}}},"inputs":{"literals":{"in1":{"scalar":{"primitive":{"stringValue":""}}},"in2":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t1\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.imperative_wf_style.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.imperative_wf_style","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t2\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.imperative_wf_style.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.imperative_wf_style","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.imperative_wf_style.t3\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.imperative_wf_style.t3"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"collectionType":{"simple":"STRING"}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.imperative_wf_style","task-name","t3"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/120_core.flyte_basics.lp.my_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/120_core.flyte_basics.lp.my_wf_2_wf_crd.json index 90cf87a3ff..220f16a8f5 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/120_core.flyte_basics.lp.my_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/120_core.flyte_basics.lp.my_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-lp-my-wf"}},"spec":{"id":"::core.flyte_basics.lp.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.lp.square\"","inputBindings":[{"var":"val","binding":{"promise":{"nodeId":"start-node","var":"val"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.lp.my_wf"},"interface":{"inputs":{"variables":{"val":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"val":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.lp.square\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.lp.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"val":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.lp","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-lp-my-wf"}},"spec":{"id":"::core.flyte_basics.lp.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"square","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.lp.square\"","inputBindings":[{"var":"val","binding":{"promise":{"nodeId":"start-node","var":"val"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.lp.my_wf"},"interface":{"inputs":{"variables":{"val":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}}},"inputs":{"literals":{"val":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.lp.square\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.lp.square"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"val":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"INTEGER"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.lp","task-name","square"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/125_core.flyte_basics.lp.go_greet_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/125_core.flyte_basics.lp.go_greet_2_wf_crd.json index 491d916111..8d77add724 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/125_core.flyte_basics.lp.go_greet_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/125_core.flyte_basics.lp.go_greet_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-lp-go-greet"}},"spec":{"id":"::core.flyte_basics.lp.go_greet","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"greet","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.lp.greet\"","inputBindings":[{"var":"am","binding":{"promise":{"nodeId":"start-node","var":"am"}}},{"var":"day_of_week","binding":{"promise":{"nodeId":"start-node","var":"day_of_week"}}},{"var":"number","binding":{"promise":{"nodeId":"start-node","var":"number"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.lp.go_greet"},"interface":{"inputs":{"variables":{"am":{"type":{"simple":"BOOLEAN"}},"day_of_week":{"type":{"simple":"STRING"}},"number":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"am":{"scalar":{"primitive":{"boolean":false}}},"day_of_week":{"scalar":{"primitive":{"stringValue":""}}},"number":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.lp.greet\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.lp.greet"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"am":{"type":{"simple":"BOOLEAN"}},"day_of_week":{"type":{"simple":"STRING"}},"number":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.lp","task-name","greet"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-lp-go-greet"}},"spec":{"id":"::core.flyte_basics.lp.go_greet","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"greet","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.lp.greet\"","inputBindings":[{"var":"am","binding":{"promise":{"nodeId":"start-node","var":"am"}}},{"var":"day_of_week","binding":{"promise":{"nodeId":"start-node","var":"day_of_week"}}},{"var":"number","binding":{"promise":{"nodeId":"start-node","var":"number"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.lp.go_greet"},"interface":{"inputs":{"variables":{"am":{"type":{"simple":"BOOLEAN"}},"day_of_week":{"type":{"simple":"STRING"}},"number":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"am":{"scalar":{"primitive":{"boolean":false}}},"day_of_week":{"scalar":{"primitive":{"stringValue":""}}},"number":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.lp.greet\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.lp.greet"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"am":{"type":{"simple":"BOOLEAN"}},"day_of_week":{"type":{"simple":"STRING"}},"number":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.lp","task-name","greet"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/129_core.flyte_basics.named_outputs.my_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/129_core.flyte_basics.named_outputs.my_wf_2_wf_crd.json index 6cef917439..7a2023f754 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/129_core.flyte_basics.named_outputs.my_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/129_core.flyte_basics.named_outputs.my_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-named-outputs-my-wf"}},"spec":{"id":"::core.flyte_basics.named_outputs.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"greet1","binding":{"promise":{"nodeId":"n0","var":"greet"}}},{"var":"greet2","binding":{"promise":{"nodeId":"n1","var":"greet"}}}]},"n0":{"id":"n0","name":"say_hello","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.named_outputs.say_hello\""},"n1":{"id":"n1","name":"say_hello","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.named_outputs.say_hello\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"n1":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["end-node"],"n1":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["start-node"]}},"outputBindings":[{"var":"greet1","binding":{"promise":{"nodeId":"n0","var":"greet"}}},{"var":"greet2","binding":{"promise":{"nodeId":"n1","var":"greet"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.named_outputs.my_wf"},"interface":{"inputs":{},"outputs":{"variables":{"greet1":{"type":{"simple":"STRING"}},"greet2":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.named_outputs.say_hello\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.named_outputs.say_hello"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"greet":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.named_outputs","task-name","say_hello"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-named-outputs-my-wf"}},"spec":{"id":"::core.flyte_basics.named_outputs.my_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"greet1","binding":{"promise":{"nodeId":"n0","var":"greet"}}},{"var":"greet2","binding":{"promise":{"nodeId":"n1","var":"greet"}}}]},"n0":{"id":"n0","name":"say_hello","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.named_outputs.say_hello\""},"n1":{"id":"n1","name":"say_hello","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.named_outputs.say_hello\""},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"n1":["end-node"],"start-node":["n0","n1"]},"edges":{"downstream":{"n0":["end-node"],"n1":["end-node"],"start-node":["n0","n1"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["start-node"]}},"outputBindings":[{"var":"greet1","binding":{"promise":{"nodeId":"n0","var":"greet"}}},{"var":"greet2","binding":{"promise":{"nodeId":"n1","var":"greet"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.named_outputs.my_wf"},"interface":{"inputs":{},"outputs":{"variables":{"greet1":{"type":{"simple":"STRING"}},"greet2":{"type":{"simple":"STRING"}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.named_outputs.say_hello\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.named_outputs.say_hello"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"greet":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.named_outputs","task-name","say_hello"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/140_core.flyte_basics.shell_task.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/140_core.flyte_basics.shell_task.wf_2_wf_crd.json index a9ac96ee24..9e1917a364 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/140_core.flyte_basics.shell_task.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/140_core.flyte_basics.shell_task.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-shell-task-wf"}},"spec":{"id":"::core.flyte_basics.shell_task.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n3","var":"k"}}}]},"n0":{"id":"n0","name":"create_entities","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.shell_task.create_entities\""},"n1":{"id":"n1","name":"task_1","resources":{},"kind":"task","task":"resource_type:TASK name:\"task_1\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"task_2","resources":{},"kind":"task","task":"resource_type:TASK name:\"task_2\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n1","var":"i"}}},{"var":"y","binding":{"promise":{"nodeId":"n0","var":"o1"}}}]},"n3":{"id":"n3","name":"task_3","resources":{},"kind":"task","task":"resource_type:TASK name:\"task_3\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"y","binding":{"promise":{"nodeId":"n0","var":"o1"}}},{"var":"z","binding":{"promise":{"nodeId":"n2","var":"j"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1","n2","n3"],"n1":["n2"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1","n2","n3"],"n1":["n2"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n3"],"n0":["start-node"],"n1":["n0"],"n2":["n0","n1"],"n3":["n0","n2"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n3","var":"k"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.shell_task.wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"blob":{}}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.shell_task.create_entities\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.shell_task.create_entities"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"blob":{}}},"o1":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","create_entities"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"task_1\"":{"id":{"resourceType":"TASK","name":"task_1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"blob":{}}}}},"outputs":{"variables":{"i":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"task_2\"":{"id":{"resourceType":"TASK","name":"task_2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"blob":{}}},"y":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}},"outputs":{"variables":{"j":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"task_3\"":{"id":{"resourceType":"TASK","name":"task_3"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"blob":{}}},"y":{"type":{"blob":{"dimensionality":"MULTIPART"}}},"z":{"type":{"blob":{}}}}},"outputs":{"variables":{"k":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","t3"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-shell-task-wf"}},"spec":{"id":"::core.flyte_basics.shell_task.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n3","var":"k"}}}]},"n0":{"id":"n0","name":"create_entities","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.shell_task.create_entities\""},"n1":{"id":"n1","name":"task_1","resources":{},"kind":"task","task":"resource_type:TASK name:\"task_1\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"task_2","resources":{},"kind":"task","task":"resource_type:TASK name:\"task_2\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n1","var":"i"}}},{"var":"y","binding":{"promise":{"nodeId":"n0","var":"o1"}}}]},"n3":{"id":"n3","name":"task_3","resources":{},"kind":"task","task":"resource_type:TASK name:\"task_3\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n0","var":"o0"}}},{"var":"y","binding":{"promise":{"nodeId":"n0","var":"o1"}}},{"var":"z","binding":{"promise":{"nodeId":"n2","var":"j"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1","n2","n3"],"n1":["n2"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1","n2","n3"],"n1":["n2"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n3"],"n0":["start-node"],"n1":["n0"],"n2":["n0","n1"],"n3":["n0","n2"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n3","var":"k"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.shell_task.wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"blob":{}}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.shell_task.create_entities\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.shell_task.create_entities"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"blob":{}}},"o1":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","create_entities"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"task_1\"":{"id":{"resourceType":"TASK","name":"task_1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"blob":{}}}}},"outputs":{"variables":{"i":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"task_2\"":{"id":{"resourceType":"TASK","name":"task_2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"blob":{}}},"y":{"type":{"blob":{"dimensionality":"MULTIPART"}}}}},"outputs":{"variables":{"j":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"task_3\"":{"id":{"resourceType":"TASK","name":"task_3"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"blob":{}}},"y":{"type":{"blob":{"dimensionality":"MULTIPART"}}},"z":{"type":{"blob":{}}}}},"outputs":{"variables":{"k":{"type":{"blob":{}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.shell_task","task-name","t3"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/147_core.flyte_basics.task_cache.cached_dataframe_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/147_core.flyte_basics.task_cache.cached_dataframe_wf_2_wf_crd.json index acb123ed74..57dbbcbaec 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/147_core.flyte_basics.task_cache.cached_dataframe_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/147_core.flyte_basics.task_cache.cached_dataframe_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-task-cache-cached-dataframe-wf"}},"spec":{"id":"::core.flyte_basics.task_cache.cached_dataframe_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"uncached_data_reading_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.uncached_data_reading_task\""},"n1":{"id":"n1","name":"cached_data_processing_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.cached_data_processing_task\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"cached_data_processing_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.cached_data_processing_task\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n3":{"id":"n3","name":"compare_dataframes","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.compare_dataframes\"","inputBindings":[{"var":"df1","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"df2","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1","n2"],"n1":["n2","n3"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1","n2"],"n1":["n2","n3"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n3"],"n0":["start-node"],"n1":["n0"],"n2":["n0","n1"],"n3":["n1","n2"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.task_cache.cached_dataframe_wf"},"interface":{"inputs":{},"outputs":{}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.task_cache.cached_data_processing_task\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.task_cache.cached_data_processing_task"},"type":"python-task","metadata":{"discoverable":true,"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{},"discoveryVersion":"1.0"},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.task_cache","task-name","cached_data_processing_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.task_cache.compare_dataframes\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.task_cache.compare_dataframes"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df1":{"type":{"structuredDatasetType":{"format":"parquet"}}},"df2":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.task_cache","task-name","compare_dataframes"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.task_cache.uncached_data_reading_task\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.task_cache.uncached_data_reading_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.task_cache","task-name","uncached_data_reading_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-flyte-basics-task-cache-cached-dataframe-wf"}},"spec":{"id":"::core.flyte_basics.task_cache.cached_dataframe_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"uncached_data_reading_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.uncached_data_reading_task\""},"n1":{"id":"n1","name":"cached_data_processing_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.cached_data_processing_task\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"cached_data_processing_task","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.cached_data_processing_task\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n3":{"id":"n3","name":"compare_dataframes","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.flyte_basics.task_cache.compare_dataframes\"","inputBindings":[{"var":"df1","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"df2","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1","n2"],"n1":["n2","n3"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1","n2"],"n1":["n2","n3"],"n2":["n3"],"n3":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n3"],"n0":["start-node"],"n1":["n0"],"n2":["n0","n1"],"n3":["n1","n2"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.flyte_basics.task_cache.cached_dataframe_wf"},"interface":{"inputs":{},"outputs":{}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.flyte_basics.task_cache.cached_data_processing_task\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.task_cache.cached_data_processing_task"},"type":"python-task","metadata":{"discoverable":true,"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{},"discoveryVersion":"1.0"},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.task_cache","task-name","cached_data_processing_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.task_cache.compare_dataframes\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.task_cache.compare_dataframes"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df1":{"type":{"structuredDatasetType":{"format":"parquet"}}},"df2":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.task_cache","task-name","compare_dataframes"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.flyte_basics.task_cache.uncached_data_reading_task\"":{"id":{"resourceType":"TASK","name":"core.flyte_basics.task_cache.uncached_data_reading_task"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.flyte_basics.task_cache","task-name","uncached_data_reading_task"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/151_core.scheduled_workflows.lp_schedules.date_formatter_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/151_core.scheduled_workflows.lp_schedules.date_formatter_wf_2_wf_crd.json index fa952a628b..4a26815368 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/151_core.scheduled_workflows.lp_schedules.date_formatter_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/151_core.scheduled_workflows.lp_schedules.date_formatter_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-scheduled-workflows-lp-schedules-date-formatter-wf"}},"spec":{"id":"::core.scheduled_workflows.lp_schedules.date_formatter_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"format_date","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.format_date\"","inputBindings":[{"var":"run_date","binding":{"promise":{"nodeId":"start-node","var":"kickoff_time"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.scheduled_workflows.lp_schedules.date_formatter_wf"},"interface":{"inputs":{"variables":{"kickoff_time":{"type":{"simple":"DATETIME"}}}},"outputs":{}}},"inputs":{"literals":{"kickoff_time":{"scalar":{"primitive":{"datetime":"1970-01-01T00:00:00.000010Z"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.format_date\"":{"id":{"resourceType":"TASK","name":"core.scheduled_workflows.lp_schedules.format_date"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"run_date":{"type":{"simple":"DATETIME"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.scheduled_workflows.lp_schedules","task-name","format_date"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-scheduled-workflows-lp-schedules-date-formatter-wf"}},"spec":{"id":"::core.scheduled_workflows.lp_schedules.date_formatter_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"format_date","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.format_date\"","inputBindings":[{"var":"run_date","binding":{"promise":{"nodeId":"start-node","var":"kickoff_time"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.scheduled_workflows.lp_schedules.date_formatter_wf"},"interface":{"inputs":{"variables":{"kickoff_time":{"type":{"simple":"DATETIME"}}}},"outputs":{}}},"inputs":{"literals":{"kickoff_time":{"scalar":{"primitive":{"datetime":"1970-01-01T00:00:00.000010Z"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.format_date\"":{"id":{"resourceType":"TASK","name":"core.scheduled_workflows.lp_schedules.format_date"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"run_date":{"type":{"simple":"DATETIME"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.scheduled_workflows.lp_schedules","task-name","format_date"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/155_core.scheduled_workflows.lp_schedules.positive_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/155_core.scheduled_workflows.lp_schedules.positive_wf_2_wf_crd.json index 9230755a09..a3909a69f5 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/155_core.scheduled_workflows.lp_schedules.positive_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/155_core.scheduled_workflows.lp_schedules.positive_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-scheduled-workflows-lp-schedules-positive-wf"}},"spec":{"id":"::core.scheduled_workflows.lp_schedules.positive_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"be_positive","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.be_positive\"","inputBindings":[{"var":"name","binding":{"promise":{"nodeId":"start-node","var":"name"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.scheduled_workflows.lp_schedules.positive_wf"},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{}}},"inputs":{"literals":{"name":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.be_positive\"":{"id":{"resourceType":"TASK","name":"core.scheduled_workflows.lp_schedules.be_positive"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.scheduled_workflows.lp_schedules","task-name","be_positive"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-scheduled-workflows-lp-schedules-positive-wf"}},"spec":{"id":"::core.scheduled_workflows.lp_schedules.positive_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end"},"n0":{"id":"n0","name":"be_positive","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.be_positive\"","inputBindings":[{"var":"name","binding":{"promise":{"nodeId":"start-node","var":"name"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"identifier":{"resourceType":"WORKFLOW","name":"core.scheduled_workflows.lp_schedules.positive_wf"},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{}}},"inputs":{"literals":{"name":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.scheduled_workflows.lp_schedules.be_positive\"":{"id":{"resourceType":"TASK","name":"core.scheduled_workflows.lp_schedules.be_positive"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.scheduled_workflows.lp_schedules","task-name","be_positive"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/162_core.type_system.custom_objects.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/162_core.type_system.custom_objects.wf_2_wf_crd.json index 4e70c6b90f..02a12f4faf 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/162_core.type_system.custom_objects.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/162_core.type_system.custom_objects.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-custom-objects-wf"}},"spec":{"id":"::core.type_system.custom_objects.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n4","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"upload_result","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.upload_result\""},"n1":{"id":"n1","name":"download_result","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.download_result\"","inputBindings":[{"var":"res","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"stringify","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.stringify\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"x"}}}]},"n3":{"id":"n3","name":"stringify","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.stringify\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"y"}}}]},"n4":{"id":"n4","name":"add","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.add\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n2","var":"o0"}}},{"var":"y","binding":{"promise":{"nodeId":"n3","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"n2":["n4"],"n3":["n4"],"n4":["end-node"],"start-node":["n0","n2","n3"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"n2":["n4"],"n3":["n4"],"n4":["end-node"],"start-node":["n0","n2","n3"]},"upstream":{"end-node":["n0","n1","n4"],"n0":["start-node"],"n1":["n0"],"n2":["start-node"],"n3":["start-node"],"n4":["n2","n3"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n4","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.custom_objects.wf"},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}},"y":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}},"o1":{"type":{"simple":"STRUCT"}}}}}},"inputs":{"literals":{"x":{"scalar":{"primitive":{"integer":"0"}}},"y":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.custom_objects.add\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.add"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"STRUCT"}},"y":{"type":{"simple":"STRUCT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","add"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.custom_objects.download_result\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.download_result"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"res":{"type":{"simple":"STRUCT"}}}},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","download_result"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.custom_objects.stringify\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.stringify"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","stringify"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.custom_objects.upload_result\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.upload_result"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","upload_result"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-custom-objects-wf"}},"spec":{"id":"::core.type_system.custom_objects.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n4","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"upload_result","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.upload_result\""},"n1":{"id":"n1","name":"download_result","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.download_result\"","inputBindings":[{"var":"res","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"stringify","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.stringify\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"x"}}}]},"n3":{"id":"n3","name":"stringify","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.stringify\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"start-node","var":"y"}}}]},"n4":{"id":"n4","name":"add","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.custom_objects.add\"","inputBindings":[{"var":"x","binding":{"promise":{"nodeId":"n2","var":"o0"}}},{"var":"y","binding":{"promise":{"nodeId":"n3","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"n2":["n4"],"n3":["n4"],"n4":["end-node"],"start-node":["n0","n2","n3"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"n2":["n4"],"n3":["n4"],"n4":["end-node"],"start-node":["n0","n2","n3"]},"upstream":{"end-node":["n0","n1","n4"],"n0":["start-node"],"n1":["n0"],"n2":["start-node"],"n3":["start-node"],"n4":["n2","n3"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n4","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.custom_objects.wf"},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}},"y":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}},"o1":{"type":{"simple":"STRUCT"}}}}}},"inputs":{"literals":{"x":{"scalar":{"primitive":{"integer":"0"}}},"y":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.custom_objects.add\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.add"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"STRUCT"}},"y":{"type":{"simple":"STRUCT"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","add"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.custom_objects.download_result\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.download_result"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"res":{"type":{"simple":"STRUCT"}}}},"outputs":{}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","download_result"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.custom_objects.stringify\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.stringify"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"x":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","stringify"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.custom_objects.upload_result\"":{"id":{"resourceType":"TASK","name":"core.type_system.custom_objects.upload_result"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"simple":"STRUCT"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.custom_objects","task-name","upload_result"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/166_core.type_system.enums.enum_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/166_core.type_system.enums.enum_wf_2_wf_crd.json index 72cee531c6..c3203432d2 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/166_core.type_system.enums.enum_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/166_core.type_system.enums.enum_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-enums-enum-wf"}},"spec":{"id":"::core.type_system.enums.enum_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"enum_stringify","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.enums.enum_stringify\"","inputBindings":[{"var":"c","binding":{"promise":{"nodeId":"start-node","var":"c"}}}]},"n1":{"id":"n1","name":"string_to_enum","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.enums.string_to_enum\"","inputBindings":[{"var":"c","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.enums.enum_wf"},"interface":{"inputs":{"variables":{"c":{"type":{"enumType":{"values":["red","green","blue"]}}}}},"outputs":{"variables":{"o0":{"type":{"enumType":{"values":["red","green","blue"]}}},"o1":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"c":{"scalar":{"primitive":{"stringValue":"red"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.enums.enum_stringify\"":{"id":{"resourceType":"TASK","name":"core.type_system.enums.enum_stringify"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"c":{"type":{"enumType":{"values":["red","green","blue"]}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.enums","task-name","enum_stringify"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.enums.string_to_enum\"":{"id":{"resourceType":"TASK","name":"core.type_system.enums.string_to_enum"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"c":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"enumType":{"values":["red","green","blue"]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.enums","task-name","string_to_enum"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-enums-enum-wf"}},"spec":{"id":"::core.type_system.enums.enum_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"enum_stringify","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.enums.enum_stringify\"","inputBindings":[{"var":"c","binding":{"promise":{"nodeId":"start-node","var":"c"}}}]},"n1":{"id":"n1","name":"string_to_enum","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.enums.string_to_enum\"","inputBindings":[{"var":"c","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node","n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0","n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}},{"var":"o1","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.enums.enum_wf"},"interface":{"inputs":{"variables":{"c":{"type":{"enumType":{"values":["red","green","blue"]}}}}},"outputs":{"variables":{"o0":{"type":{"enumType":{"values":["red","green","blue"]}}},"o1":{"type":{"simple":"STRING"}}}}}},"inputs":{"literals":{"c":{"scalar":{"primitive":{"stringValue":"red"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.enums.enum_stringify\"":{"id":{"resourceType":"TASK","name":"core.type_system.enums.enum_stringify"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"c":{"type":{"enumType":{"values":["red","green","blue"]}}}}},"outputs":{"variables":{"o0":{"type":{"simple":"STRING"}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.enums","task-name","enum_stringify"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.enums.string_to_enum\"":{"id":{"resourceType":"TASK","name":"core.type_system.enums.string_to_enum"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"c":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"enumType":{"values":["red","green","blue"]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.enums","task-name","string_to_enum"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/169_core.type_system.flyte_pickle.welcome_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/169_core.type_system.flyte_pickle.welcome_2_wf_crd.json index fe364ef853..adee50a30b 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/169_core.type_system.flyte_pickle.welcome_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/169_core.type_system.flyte_pickle.welcome_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-flyte-pickle-welcome"}},"spec":{"id":"::core.type_system.flyte_pickle.welcome","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"greet","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.flyte_pickle.greet\"","inputBindings":[{"var":"name","binding":{"promise":{"nodeId":"start-node","var":"name"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.flyte_pickle.welcome"},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"format":"PythonPickle"}}}}}}},"inputs":{"literals":{"name":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.flyte_pickle.greet\"":{"id":{"resourceType":"TASK","name":"core.type_system.flyte_pickle.greet"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"format":"PythonPickle"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.flyte_pickle","task-name","greet"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-flyte-pickle-welcome"}},"spec":{"id":"::core.type_system.flyte_pickle.welcome","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n0":{"id":"n0","name":"greet","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.flyte_pickle.greet\"","inputBindings":[{"var":"name","binding":{"promise":{"nodeId":"start-node","var":"name"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n0"],"n0":["start-node"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n0","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.flyte_pickle.welcome"},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"format":"PythonPickle"}}}}}}},"inputs":{"literals":{"name":{"scalar":{"primitive":{"stringValue":""}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.flyte_pickle.greet\"":{"id":{"resourceType":"TASK","name":"core.type_system.flyte_pickle.greet"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"name":{"type":{"simple":"STRING"}}}},"outputs":{"variables":{"o0":{"type":{"blob":{"format":"PythonPickle"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.flyte_pickle","task-name","greet"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/173_core.type_system.schema.df_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/173_core.type_system.schema.df_wf_2_wf_crd.json index 2c037ab054..1213d7b00c 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/173_core.type_system.schema.df_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/173_core.type_system.schema.df_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-schema-df-wf"}},"spec":{"id":"::core.type_system.schema.df_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"get_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.schema.get_df\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"add_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.schema.add_df\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.schema.df_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.schema.add_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.schema.add_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.schema","task-name","add_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.schema.get_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.schema.get_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.schema","task-name","get_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-schema-df-wf"}},"spec":{"id":"::core.type_system.schema.df_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"get_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.schema.get_df\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"add_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.schema.add_df\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.schema.df_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.schema.add_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.schema.add_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.schema","task-name","add_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.schema.get_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.schema.get_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.schema","task-name","get_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/179_core.type_system.structured_dataset.pandas_compatibility_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/179_core.type_system.structured_dataset.pandas_compatibility_wf_2_wf_crd.json index cd08c5854b..4fe62d0323 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/179_core.type_system.structured_dataset.pandas_compatibility_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/179_core.type_system.structured_dataset.pandas_compatibility_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-structured-dataset-pandas-compatibility-wf"}},"spec":{"id":"::core.type_system.structured_dataset.pandas_compatibility_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"get_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_df\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"get_subset_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"to_numpy","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"","inputBindings":[{"var":"ds","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n2"],"n0":["start-node"],"n1":["n0"],"n2":["n1"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.structured_dataset.pandas_compatibility_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.structured_dataset.get_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Name","literalType":{"simple":"STRING"}},{"name":"Age","literalType":{"simple":"INTEGER"}},{"name":"Height","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_subset_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_subset_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.to_numpy"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"ds":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","to_numpy"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-structured-dataset-pandas-compatibility-wf"}},"spec":{"id":"::core.type_system.structured_dataset.pandas_compatibility_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"get_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_df\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"get_subset_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"to_numpy","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"","inputBindings":[{"var":"ds","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n2"],"n0":["start-node"],"n1":["n0"],"n2":["n1"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.structured_dataset.pandas_compatibility_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.structured_dataset.get_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Name","literalType":{"simple":"STRING"}},{"name":"Age","literalType":{"simple":"INTEGER"}},{"name":"Height","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_subset_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_subset_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.to_numpy"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"ds":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","to_numpy"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/181_core.type_system.structured_dataset.schema_compatibility_wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/181_core.type_system.structured_dataset.schema_compatibility_wf_2_wf_crd.json index d59af56d46..926ef79789 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/181_core.type_system.structured_dataset.schema_compatibility_wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/181_core.type_system.structured_dataset.schema_compatibility_wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-structured-dataset-schema-compatibility-wf"}},"spec":{"id":"::core.type_system.structured_dataset.schema_compatibility_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"get_schema_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_schema_df\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"get_subset_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"to_numpy","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"","inputBindings":[{"var":"ds","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n2"],"n0":["start-node"],"n1":["n0"],"n2":["n1"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.structured_dataset.schema_compatibility_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.structured_dataset.get_schema_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_schema_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"Name","type":"STRING"},{"name":"Age"},{"name":"Height"}]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_schema_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_subset_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_subset_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.to_numpy"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"ds":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","to_numpy"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-structured-dataset-schema-compatibility-wf"}},"spec":{"id":"::core.type_system.structured_dataset.schema_compatibility_wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}]},"n0":{"id":"n0","name":"get_schema_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_schema_df\"","inputBindings":[{"var":"a","binding":{"promise":{"nodeId":"start-node","var":"a"}}}]},"n1":{"id":"n1","name":"get_subset_df","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"","inputBindings":[{"var":"df","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"n2":{"id":"n2","name":"to_numpy","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"","inputBindings":[{"var":"ds","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["n2"],"n2":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n2"],"n0":["start-node"],"n1":["n0"],"n2":["n1"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n2","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.structured_dataset.schema_compatibility_wf"},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}}},"inputs":{"literals":{"a":{"scalar":{"primitive":{"integer":"0"}}}}},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.structured_dataset.get_schema_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_schema_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"a":{"type":{"simple":"INTEGER"}}}},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"Name","type":"STRING"},{"name":"Age"},{"name":"Height"}]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_schema_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.get_subset_df\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.get_subset_df"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"df":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","get_subset_df"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.structured_dataset.to_numpy\"":{"id":{"resourceType":"TASK","name":"core.type_system.structured_dataset.to_numpy"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"ds":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}},"outputs":{"variables":{"o0":{"type":{"structuredDatasetType":{"columns":[{"name":"Age","literalType":{"simple":"INTEGER"}}],"format":"parquet"}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.structured_dataset","task-name","to_numpy"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/185_core.type_system.typed_schema.wf_2_wf_crd.json b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/185_core.type_system.typed_schema.wf_2_wf_crd.json index fbe7aa0250..15de00052d 100755 --- a/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/185_core.type_system.typed_schema.wf_2_wf_crd.json +++ b/flytepropeller/pkg/compiler/test/testdata/snacks-core/k8s/185_core.type_system.typed_schema.wf_2_wf_crd.json @@ -1 +1 @@ -{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-typed-schema-wf"}},"spec":{"id":"::core.type_system.typed_schema.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.typed_schema.t1\""},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.typed_schema.t2\"","inputBindings":[{"var":"schema","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.typed_schema.wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"x"}]}}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.typed_schema.t1\"":{"id":{"resourceType":"TASK","name":"core.type_system.typed_schema.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"x"},{"name":"y","type":"STRING"}]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.typed_schema","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.typed_schema.t2\"":{"id":{"resourceType":"TASK","name":"core.type_system.typed_schema.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"schema":{"type":{"schema":{"columns":[{"name":"x"},{"name":"y","type":"STRING"}]}}}}},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"x"}]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.typed_schema","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null}} \ No newline at end of file +{"kind":"flyteworkflow","apiVersion":"flyte.lyft.com/v1alpha1","metadata":{"name":"name","namespace":"namespace","creationTimestamp":null,"labels":{"domain":"domain","execution-id":"name","project":"hello","shard-key":"6","workflow-name":"core-type-system-typed-schema-wf"}},"spec":{"id":"::core.type_system.typed_schema.wf","nodes":{"end-node":{"id":"end-node","resources":{},"kind":"end","inputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}]},"n0":{"id":"n0","name":"t1","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.typed_schema.t1\""},"n1":{"id":"n1","name":"t2","resources":{},"kind":"task","task":"resource_type:TASK name:\"core.type_system.typed_schema.t2\"","inputBindings":[{"var":"schema","binding":{"promise":{"nodeId":"n0","var":"o0"}}}]},"start-node":{"id":"start-node","resources":{},"kind":"start"}},"connections":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"edges":{"downstream":{"n0":["n1"],"n1":["end-node"],"start-node":["n0"]},"upstream":{"end-node":["n1"],"n0":["start-node"],"n1":["n0"]}},"outputBindings":[{"var":"o0","binding":{"promise":{"nodeId":"n1","var":"o0"}}}],"identifier":{"resourceType":"WORKFLOW","name":"core.type_system.typed_schema.wf"},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"x"}]}}}}}}},"inputs":{},"executionId":{},"tasks":{"resource_type:TASK name:\"core.type_system.typed_schema.t1\"":{"id":{"resourceType":"TASK","name":"core.type_system.typed_schema.t1"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"x"},{"name":"y","type":"STRING"}]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.typed_schema","task-name","t1"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}},"resource_type:TASK name:\"core.type_system.typed_schema.t2\"":{"id":{"resourceType":"TASK","name":"core.type_system.typed_schema.t2"},"type":"python-task","metadata":{"runtime":{"type":"FLYTE_SDK","version":"0.32.6","flavor":"python"},"retries":{}},"interface":{"inputs":{"variables":{"schema":{"type":{"schema":{"columns":[{"name":"x"},{"name":"y","type":"STRING"}]}}}}},"outputs":{"variables":{"o0":{"type":{"schema":{"columns":[{"name":"x"}]}}}}}},"container":{"image":"ghcr.io/flyteorg/flytecookbook:core-8b8e1a849c9adfca88049a074b10dad278f70077","args":["pyflyte-execute","--inputs","{{.input}}","--output-prefix","{{.outputPrefix}}","--raw-output-data-prefix","{{.rawOutputDataPrefix}}","--checkpoint-path","{{.checkpointOutputPrefix}}","--prev-checkpoint","{{.prevCheckpointPrefix}}","--resolver","flytekit.core.python_auto_container.default_task_resolver","--","task-module","core.type_system.typed_schema","task-name","t2"],"resources":{},"config":[{"key":"testKey1","value":"testValue1"},{"key":"testKey2","value":"testValue2"},{"key":"testKey3","value":"testValue3"}]}}},"node-defaults":{},"securityContext":{},"status":{"phase":0},"rawOutputDataConfig":{},"executionConfig":{"TaskPluginImpls":null,"MaxParallelism":0,"RecoveryExecution":{},"TaskResources":{"Requests":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"},"Limits":{"CPU":"0","Memory":"0","EphemeralStorage":"0","Storage":"0","GPU":"0"}},"Interruptible":null,"OverwriteCache":false,"EnvironmentVariables":null,"ExecutionEnvAssignments":null}} \ No newline at end of file diff --git a/flytepropeller/pkg/controller/controller.go b/flytepropeller/pkg/controller/controller.go index 4e87d63d5d..b7e31f3f07 100644 --- a/flytepropeller/pkg/controller/controller.go +++ b/flytepropeller/pkg/controller/controller.go @@ -438,14 +438,20 @@ func New(ctx context.Context, cfg *config.Config, kubeClientset kubernetes.Inter return nil, errors.Wrapf(err, "failed to create node handler factory") } + executionEnvClient, err := NewExecutionEnvClient(ctx, kubeClient, scope) + if err != nil { + return nil, errors.Wrapf(err, "failed to create execution environment client") + } + nodeExecutor, err := nodes.NewExecutor(ctx, cfg.NodeConfig, store, controller.enqueueWorkflowForNodeUpdates, eventSink, launchPlanActor, launchPlanActor, cfg.MaxDatasetSizeBytes, storage.DataReference(cfg.DefaultRawOutputPrefix), kubeClient, - cacheClient, recoveryClient, &cfg.EventConfig, cfg.ClusterID, signalClient, nodeHandlerFactory, scope) + cacheClient, recoveryClient, &cfg.EventConfig, cfg.ClusterID, signalClient, nodeHandlerFactory, executionEnvClient, scope) if err != nil { return nil, errors.Wrapf(err, "Failed to create Controller.") } - workflowExecutor, err := workflow.NewExecutor(ctx, store, controller.enqueueWorkflowForNodeUpdates, eventSink, controller.recorder, cfg.MetadataPrefix, nodeExecutor, &cfg.EventConfig, cfg.ClusterID, scope) + workflowExecutor, err := workflow.NewExecutor(ctx, store, controller.enqueueWorkflowForNodeUpdates, eventSink, + controller.recorder, cfg.MetadataPrefix, nodeExecutor, &cfg.EventConfig, cfg.ClusterID, executionEnvClient, scope) if err != nil { return nil, err } diff --git a/flytepropeller/pkg/controller/execution_env.go b/flytepropeller/pkg/controller/execution_env.go new file mode 100644 index 0000000000..7a1da9b76e --- /dev/null +++ b/flytepropeller/pkg/controller/execution_env.go @@ -0,0 +1,44 @@ +package controller + +import ( + "context" + + _struct "github.com/golang/protobuf/ptypes/struct" + "github.com/unionai/flyte/fasttask/plugin" + + pluginscore "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" + "github.com/flyteorg/flyte/flytepropeller/pkg/controller/executors" + "github.com/flyteorg/flyte/flytestdlib/promutils" +) + +// TODO currently we only have one ExecutionEnvironment builder (ie. fast-task), but in the future +// we plan to have more. this code should be minorly refactored to reflect that support. +type ExecutionEnvClient struct { + envBuilders map[string]*plugin.InMemoryEnvBuilder +} + +// Create initializes an execution environment with the given ID and spec. +func (e *ExecutionEnvClient) Create(ctx context.Context, executionEnvID string, executionEnvSpec *_struct.Struct) (*_struct.Struct, error) { + fastTaskEnvBuilder := e.envBuilders["fast-task"] + return fastTaskEnvBuilder.Create(ctx, executionEnvID, executionEnvSpec) +} + +// Get returns the execution environment with the given ID. +func (e *ExecutionEnvClient) Get(ctx context.Context, executionEnvID string) *_struct.Struct { + fastTaskEnvBuilder := e.envBuilders["fast-task"] + return fastTaskEnvBuilder.Get(ctx, executionEnvID) +} + +// NewExecutionEnvClient creates a new ExecutionEnvClient. +func NewExecutionEnvClient(ctx context.Context, kubeClient executors.Client, scope promutils.Scope) (pluginscore.ExecutionEnvClient, error) { + fastTaskEnvBuilder := plugin.NewEnvironmentBuilder(kubeClient) + if err := fastTaskEnvBuilder.Start(ctx); err != nil { + return nil, err + } + + return &ExecutionEnvClient{ + envBuilders: map[string]*plugin.InMemoryEnvBuilder{ + "fast-task": fastTaskEnvBuilder, + }, + }, nil +} diff --git a/flytepropeller/pkg/controller/nodes/array/handler.go b/flytepropeller/pkg/controller/nodes/array/handler.go index 991a28c1c3..9a79b1fffd 100644 --- a/flytepropeller/pkg/controller/nodes/array/handler.go +++ b/flytepropeller/pkg/controller/nodes/array/handler.go @@ -505,7 +505,7 @@ func (a *arrayNodeHandler) Handle(ctx context.Context, nCtx interfaces.NodeExecu } } case v1alpha1.NodeKindWorkflow: - // TODO - to support launchplans we will need to process the output interface variables here + // TODO - to support launchplans we will need to process the output interface variables here fallthrough default: logger.Warnf(ctx, "ArrayNode does not support pre-populating outputLiteral collections for node kind '%s'", arrayNode.GetSubNodeSpec().GetKind()) diff --git a/flytepropeller/pkg/controller/nodes/array/handler_test.go b/flytepropeller/pkg/controller/nodes/array/handler_test.go index f0b91217e0..5f4445a8e3 100644 --- a/flytepropeller/pkg/controller/nodes/array/handler_test.go +++ b/flytepropeller/pkg/controller/nodes/array/handler_test.go @@ -12,7 +12,8 @@ import ( idlcore "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" - pluginmocks "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/io/mocks" + plugincoremocks "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core/mocks" + pluginiomocks "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/io/mocks" eventmocks "github.com/flyteorg/flyte/flytepropeller/events/mocks" "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" "github.com/flyteorg/flyte/flytepropeller/pkg/controller/config" @@ -56,11 +57,12 @@ func createArrayNodeHandler(ctx context.Context, t *testing.T, nodeHandler inter mockKubeClient := execmocks.NewFakeKubeClient() mockRecoveryClient := &recoverymocks.Client{} mockSignalClient := &gatemocks.SignalServiceClient{} + mockExecutionEnvClient := &plugincoremocks.ExecutionEnvClient{} noopCatalogClient := catalog.NOOPCatalog{} // create node executor - nodeExecutor, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, dataStore, enqueueWorkflowFunc, mockEventSink, adminClient, - adminClient, 10, "s3://bucket/", mockKubeClient, noopCatalogClient, mockRecoveryClient, eventConfig, "clusterID", mockSignalClient, mockHandlerFactory, scope) + nodeExecutor, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, dataStore, enqueueWorkflowFunc, mockEventSink, adminClient, adminClient, 10, + "s3://bucket/", mockKubeClient, noopCatalogClient, mockRecoveryClient, eventConfig, "clusterID", mockSignalClient, mockHandlerFactory, mockExecutionEnvClient, scope) assert.NoError(t, err) // return ArrayNodeHandler @@ -126,7 +128,7 @@ func createNodeExecutionContext(dataStore *storage.DataStore, eventRecorder inte nCtx.OnEventsRecorder().Return(eventRecorder) // InputReader - inputFilePaths := &pluginmocks.InputFilePaths{} + inputFilePaths := &pluginiomocks.InputFilePaths{} inputFilePaths.OnGetInputPath().Return(storage.DataReference("s3://bucket/input")) nCtx.OnInputReader().Return( newStaticInputReader( diff --git a/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go b/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go index 8d8604535e..f22ed86c15 100644 --- a/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go +++ b/flytepropeller/pkg/controller/nodes/dynamic/dynamic_workflow_test.go @@ -509,7 +509,7 @@ func Test_dynamicNodeHandler_buildContextualDynamicWorkflow_withLaunchPlans(t *t composedPBStore.OnWriteRawMatch( mock.MatchedBy(func(ctx context.Context) bool { return true }), storage.DataReference("s3://my-s3-bucket/foo/bar/futures_compiled.pb"), - int64(1892), + int64(1923), storage.Options{}, mock.MatchedBy(func(rdr *bytes.Reader) bool { return true })).Return(errors.New("foo")) composedPBStore.OnWriteProtobufMatch( diff --git a/flytepropeller/pkg/controller/nodes/executor.go b/flytepropeller/pkg/controller/nodes/executor.go index f0ebc3eb59..9752839eb4 100644 --- a/flytepropeller/pkg/controller/nodes/executor.go +++ b/flytepropeller/pkg/controller/nodes/executor.go @@ -33,6 +33,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/service" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/catalog" + pluginscore "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flyte/flytepropeller/events" eventsErr "github.com/flyteorg/flyte/flytepropeller/events/errors" @@ -483,6 +484,7 @@ type nodeExecutor struct { defaultExecutionDeadline time.Duration enqueueWorkflow v1alpha1.EnqueueWorkflow eventConfig *config.EventConfig + executionEnvClient pluginscore.ExecutionEnvClient interruptibleFailureThreshold int32 maxDatasetSizeBytes int64 maxNodeRetriesForSystemFailures uint32 @@ -1445,7 +1447,7 @@ func replaceRemotePrefix(ctx context.Context, s string) string { func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *storage.DataStore, enQWorkflow v1alpha1.EnqueueWorkflow, eventSink events.EventSink, workflowLauncher launchplan.Executor, launchPlanReader launchplan.Reader, maxDatasetSize int64, defaultRawOutputPrefix storage.DataReference, kubeClient executors.Client, cacheClient catalog.Client, recoveryClient recovery.Client, eventConfig *config.EventConfig, clusterID string, signalClient service.SignalServiceClient, - nodeHandlerFactory interfaces.HandlerFactory, scope promutils.Scope) (interfaces.Node, error) { + nodeHandlerFactory interfaces.HandlerFactory, executionEnvClient pluginscore.ExecutionEnvClient, scope promutils.Scope) (interfaces.Node, error) { // TODO we may want to make this configurable. shardSelector, err := ioutils.NewBase36PrefixShardSelector(ctx) @@ -1496,6 +1498,7 @@ func NewExecutor(ctx context.Context, nodeConfig config.NodeConfig, store *stora defaultExecutionDeadline: nodeConfig.DefaultDeadlines.DefaultNodeExecutionDeadline.Duration, enqueueWorkflow: enQWorkflow, eventConfig: eventConfig, + executionEnvClient: executionEnvClient, interruptibleFailureThreshold: nodeConfig.InterruptibleFailureThreshold, maxDatasetSizeBytes: maxDatasetSize, maxNodeRetriesForSystemFailures: uint32(nodeConfig.MaxNodeRetriesOnSystemFailures), diff --git a/flytepropeller/pkg/controller/nodes/executor_test.go b/flytepropeller/pkg/controller/nodes/executor_test.go index e01c121806..4a63bef96d 100644 --- a/flytepropeller/pkg/controller/nodes/executor_test.go +++ b/flytepropeller/pkg/controller/nodes/executor_test.go @@ -23,6 +23,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" pluginscatalog "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/catalog" catalogmocks "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/catalog/mocks" + pluginsmocks "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core/mocks" mocks3 "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/io/mocks" "github.com/flyteorg/flyte/flytepropeller/events" eventsErr "github.com/flyteorg/flyte/flytepropeller/events/errors" @@ -51,6 +52,7 @@ var fakeKubeClient = mocks4.NewFakeKubeClient() var catalogClient = catalog.NOOPCatalog{} var recoveryClient = &recoveryMocks.Client{} var signalClient = &gatemocks.SignalServiceClient{} +var executionEnvClient = &pluginsmocks.ExecutionEnvClient{} const taskID = "tID" const inputsPath = "inputs.pb" @@ -70,8 +72,8 @@ func TestSetInputsForStartNode(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - exec, err := NewExecutor(ctx, config.GetConfig().NodeConfig, mockStorage, enQWf, eventMocks.NewMockEventSink(), adminClient, - adminClient, 10, "s3://bucket/", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + exec, err := NewExecutor(ctx, config.GetConfig().NodeConfig, mockStorage, enQWf, eventMocks.NewMockEventSink(), adminClient, adminClient, 10, "s3://bucket/", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) inputs := &core.LiteralMap{ Literals: map[string]*core.Literal{ @@ -117,8 +119,8 @@ func TestSetInputsForStartNode(t *testing.T) { }) failStorage := createFailingDatastore(t, testScope.NewSubScope("failing")) - execFail, err := NewExecutor(ctx, config.GetConfig().NodeConfig, failStorage, enQWf, eventMocks.NewMockEventSink(), adminClient, - adminClient, 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execFail, err := NewExecutor(ctx, config.GetConfig().NodeConfig, failStorage, enQWf, eventMocks.NewMockEventSink(), adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) t.Run("StorageFailure", func(t *testing.T) { w := createDummyBaseWorkflow(mockStorage) @@ -146,8 +148,8 @@ func TestNodeExecutor_Initialize(t *testing.T) { hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, memStore, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, memStore, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -158,8 +160,8 @@ func TestNodeExecutor_Initialize(t *testing.T) { hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("error")) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, memStore, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, memStore, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -178,8 +180,8 @@ func TestNodeExecutor_RecursiveNodeHandler_RecurseStartNodes(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -284,8 +286,8 @@ func TestNodeExecutor_RecursiveNodeHandler_RecurseEndNode(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -697,8 +699,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Recurse(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() nodeConfig := config.GetConfig().NodeConfig nodeConfig.EnableCRDebugMetadata = test.enableCRDebugMetadata - execIface, err := NewExecutor(ctx, nodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, nodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -772,8 +774,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Recurse(t *testing.T) { store := createInmemoryDataStore(t, promutils.NewTestScope()) adminClient := launchplan.NewFailFastLaunchPlanExecutor() - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -886,8 +888,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Recurse(t *testing.T) { hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) store := createInmemoryDataStore(t, promutils.NewTestScope()) adminClient := launchplan.NewFailFastLaunchPlanExecutor() - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -953,8 +955,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Recurse(t *testing.T) { hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) store := createInmemoryDataStore(t, promutils.NewTestScope()) adminClient := launchplan.NewFailFastLaunchPlanExecutor() - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -984,8 +986,8 @@ func TestNodeExecutor_RecursiveNodeHandler_Recurse(t *testing.T) { hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) store := createInmemoryDataStore(t, promutils.NewTestScope()) adminClient := launchplan.NewFailFastLaunchPlanExecutor() - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -1019,8 +1021,8 @@ func TestNodeExecutor_RecursiveNodeHandler_NoDownstream(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -1132,8 +1134,8 @@ func TestNodeExecutor_RecursiveNodeHandler_UpstreamNotReady(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -1250,8 +1252,8 @@ func TestNodeExecutor_RecursiveNodeHandler_BranchNode(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) // Node not yet started @@ -1856,8 +1858,8 @@ func TestNodeExecutor_RecursiveNodeHandler_ParallelismLimit(t *testing.T) { adminClient := launchplan.NewFailFastLaunchPlanExecutor() hf := &nodemocks.HandlerFactory{} hf.On("Setup", mock.Anything, mock.Anything, mock.Anything).Return(nil) - execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, - 10, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, promutils.NewTestScope()) + execIface, err := NewExecutor(ctx, config.GetConfig().NodeConfig, store, enQWf, mockEventSink, adminClient, adminClient, 10, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, hf, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) exec := execIface.(*recursiveNodeExecutor) @@ -2631,7 +2633,7 @@ func TestNodeExecutor_RecursiveNodeHandler_Cache(t *testing.T) { mockHandlerFactory.OnGetHandler(v1alpha1.NodeKindTask).Return(mockHandler, nil) nodeExecutor, err := NewExecutor(ctx, nodeConfig, dataStore, enqueueWorkflow, mockEventSink, adminClient, adminClient, maxDatasetSize, rawOutputPrefix, fakeKubeClient, catalogClient, - recoveryClient, eventConfig, testClusterID, signalClient, mockHandlerFactory, testScope) + recoveryClient, eventConfig, testClusterID, signalClient, mockHandlerFactory, executionEnvClient, testScope) assert.NoError(t, err) return nodeExecutor diff --git a/flytepropeller/pkg/controller/nodes/interfaces/mocks/node_execution_context.go b/flytepropeller/pkg/controller/nodes/interfaces/mocks/node_execution_context.go index 448b5aa13f..cf93f73fa6 100644 --- a/flytepropeller/pkg/controller/nodes/interfaces/mocks/node_execution_context.go +++ b/flytepropeller/pkg/controller/nodes/interfaces/mocks/node_execution_context.go @@ -3,7 +3,9 @@ package mocks import ( + core "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" executors "github.com/flyteorg/flyte/flytepropeller/pkg/controller/executors" + interfaces "github.com/flyteorg/flyte/flytepropeller/pkg/controller/nodes/interfaces" io "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/io" @@ -224,6 +226,40 @@ func (_m *NodeExecutionContext) ExecutionContext() executors.ExecutionContext { return r0 } +type NodeExecutionContext_GetExecutionEnvClient struct { + *mock.Call +} + +func (_m NodeExecutionContext_GetExecutionEnvClient) Return(_a0 core.ExecutionEnvClient) *NodeExecutionContext_GetExecutionEnvClient { + return &NodeExecutionContext_GetExecutionEnvClient{Call: _m.Call.Return(_a0)} +} + +func (_m *NodeExecutionContext) OnGetExecutionEnvClient() *NodeExecutionContext_GetExecutionEnvClient { + c_call := _m.On("GetExecutionEnvClient") + return &NodeExecutionContext_GetExecutionEnvClient{Call: c_call} +} + +func (_m *NodeExecutionContext) OnGetExecutionEnvClientMatch(matchers ...interface{}) *NodeExecutionContext_GetExecutionEnvClient { + c_call := _m.On("GetExecutionEnvClient", matchers...) + return &NodeExecutionContext_GetExecutionEnvClient{Call: c_call} +} + +// GetExecutionEnvClient provides a mock function with given fields: +func (_m *NodeExecutionContext) GetExecutionEnvClient() core.ExecutionEnvClient { + ret := _m.Called() + + var r0 core.ExecutionEnvClient + if rf, ok := ret.Get(0).(func() core.ExecutionEnvClient); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(core.ExecutionEnvClient) + } + } + + return r0 +} + type NodeExecutionContext_InputReader struct { *mock.Call } diff --git a/flytepropeller/pkg/controller/nodes/interfaces/node_exec_context.go b/flytepropeller/pkg/controller/nodes/interfaces/node_exec_context.go index 8f10becca4..d5d5061fd3 100644 --- a/flytepropeller/pkg/controller/nodes/interfaces/node_exec_context.go +++ b/flytepropeller/pkg/controller/nodes/interfaces/node_exec_context.go @@ -7,6 +7,7 @@ import ( "k8s.io/apimachinery/pkg/types" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" + pluginscore "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/io" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flyte/flytepropeller/events" @@ -70,4 +71,7 @@ type NodeExecutionContext interface { // TODO We should not need to pass NodeStatus, we probably only need it for DataDir, which should actually be sent using an OutputWriter interface // Deprecated NodeStatus() v1alpha1.ExecutableNodeStatus + + // GetExecutionEnvClient returns the execution environment client. + GetExecutionEnvClient() pluginscore.ExecutionEnvClient } diff --git a/flytepropeller/pkg/controller/nodes/node_exec_context.go b/flytepropeller/pkg/controller/nodes/node_exec_context.go index f42f8b0324..736540e963 100644 --- a/flytepropeller/pkg/controller/nodes/node_exec_context.go +++ b/flytepropeller/pkg/controller/nodes/node_exec_context.go @@ -5,13 +5,18 @@ import ( "fmt" "strconv" + "slices" + + _struct "github.com/golang/protobuf/ptypes/struct" "github.com/pkg/errors" "k8s.io/apimachinery/pkg/types" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" + pluginscore "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/io" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/ioutils" + pluginsutils "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/utils" "github.com/flyteorg/flyte/flytepropeller/events" eventsErr "github.com/flyteorg/flyte/flytepropeller/events/errors" "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" @@ -133,6 +138,7 @@ type nodeExecContext struct { shardSelector ioutils.ShardSelector nl executors.NodeLookup ic executors.ExecutionContext + executionEnvClient pluginscore.ExecutionEnvClient } func (e nodeExecContext) ExecutionContext() executors.ExecutionContext { @@ -203,10 +209,16 @@ func (e nodeExecContext) MaxDatasetSizeBytes() int64 { return e.maxDatasetSizeBytes } +// GetExecutionEnvClient returns the execution environment client. +func (e nodeExecContext) GetExecutionEnvClient() pluginscore.ExecutionEnvClient { + return e.executionEnvClient +} + func newNodeExecContext(_ context.Context, store *storage.DataStore, execContext executors.ExecutionContext, nl executors.NodeLookup, node v1alpha1.ExecutableNode, nodeStatus v1alpha1.ExecutableNodeStatus, inputs io.InputReader, interruptible bool, interruptibleFailureThreshold int32, maxDatasetSize int64, taskEventRecorder events.TaskEventRecorder, nodeEventRecorder events.NodeEventRecorder, tr interfaces.TaskReader, nsm *nodeStateManager, - enqueueOwner func() error, rawOutputPrefix storage.DataReference, outputShardSelector ioutils.ShardSelector) *nodeExecContext { + enqueueOwner func() error, rawOutputPrefix storage.DataReference, outputShardSelector ioutils.ShardSelector, + executionEnvClient pluginscore.ExecutionEnvClient) *nodeExecContext { md := nodeExecMetadata{ Meta: execContext, @@ -248,6 +260,7 @@ func newNodeExecContext(_ context.Context, store *storage.DataStore, execContext shardSelector: outputShardSelector, nl: nl, ic: execContext, + executionEnvClient: executionEnvClient, } } @@ -275,7 +288,25 @@ func (c *nodeExecutor) BuildNodeExecutionContext(ctx context.Context, executionC if err != nil { return nil, err } - tr = taskReader{TaskTemplate: tk.CoreTask()} + taskTemplate := tk.CoreTask() + + // if this task has been specifically assigned an execution environment, we override the + // necessary fields to ensure desired execution. + for _, executionEnvAssignment := range executionContext.GetExecutionConfig().ExecutionEnvAssignments { + if slices.Contains(executionEnvAssignment.NodeIds, currentNodeID) { + taskTemplateCopy := *taskTemplate + + environment := &_struct.Struct{} + if err := pluginsutils.MarshalStruct(executionEnvAssignment.ExecutionEnv, environment); err != nil { + return nil, fmt.Errorf("unable to marshal ExecutionEnv [%v], Err: [%v]", executionEnvAssignment.ExecutionEnv, err.Error()) + } + taskTemplate.Custom = environment + taskTemplateCopy.Type = executionEnvAssignment.GetTaskType() + + taskTemplate = &taskTemplateCopy + } + } + tr = taskReader{TaskTemplate: taskTemplate} } workflowEnqueuer := func() error { @@ -339,5 +370,6 @@ func (c *nodeExecutor) BuildNodeExecutionContext(ctx context.Context, executionC workflowEnqueuer, rawOutputPrefix, c.shardSelector, + c.executionEnvClient, ), nil } diff --git a/flytepropeller/pkg/controller/nodes/node_exec_context_test.go b/flytepropeller/pkg/controller/nodes/node_exec_context_test.go index e64abb4c99..36635d6b46 100644 --- a/flytepropeller/pkg/controller/nodes/node_exec_context_test.go +++ b/flytepropeller/pkg/controller/nodes/node_exec_context_test.go @@ -107,7 +107,7 @@ func Test_NodeContext(t *testing.T) { s, _ := storage.NewDataStore(&storage.Config{Type: storage.TypeMemory}, promutils.NewTestScope()) p := parentInfo{} execContext := executors.NewExecutionContext(w1, nil, nil, p, nil) - nCtx := newNodeExecContext(context.TODO(), s, execContext, w1, getTestNodeSpec(nil), nil, nil, false, 0, 2, nil, nil, TaskReader{}, nil, nil, "s3://bucket", ioutils.NewConstantShardSelector([]string{"x"})) + nCtx := newNodeExecContext(context.TODO(), s, execContext, w1, getTestNodeSpec(nil), nil, nil, false, 0, 2, nil, nil, TaskReader{}, nil, nil, "s3://bucket", ioutils.NewConstantShardSelector([]string{"x"}), nil) assert.Equal(t, "id", nCtx.NodeExecutionMetadata().GetLabels()["node-id"]) assert.Equal(t, "false", nCtx.NodeExecutionMetadata().GetLabels()["interruptible"]) assert.Equal(t, "task-name", nCtx.NodeExecutionMetadata().GetLabels()["task-name"]) diff --git a/flytepropeller/pkg/controller/workflow/executor.go b/flytepropeller/pkg/controller/workflow/executor.go index 355681a746..2a5bdaa36f 100644 --- a/flytepropeller/pkg/controller/workflow/executor.go +++ b/flytepropeller/pkg/controller/workflow/executor.go @@ -10,6 +10,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/core" "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" + pluginscore "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" "github.com/flyteorg/flyte/flytepropeller/events" eventsErr "github.com/flyteorg/flyte/flytepropeller/events/errors" "github.com/flyteorg/flyte/flytepropeller/pkg/apis/flyteworkflow/v1alpha1" @@ -59,15 +60,16 @@ func StatusFailed(err *core.ExecutionError) Status { } type workflowExecutor struct { - enqueueWorkflow v1alpha1.EnqueueWorkflow - store *storage.DataStore - wfRecorder events.WorkflowEventRecorder - k8sRecorder record.EventRecorder - metadataPrefix storage.DataReference - nodeExecutor interfaces.Node - metrics *workflowMetrics - eventConfig *config.EventConfig - clusterID string + enqueueWorkflow v1alpha1.EnqueueWorkflow + store *storage.DataStore + wfRecorder events.WorkflowEventRecorder + k8sRecorder record.EventRecorder + metadataPrefix storage.DataReference + nodeExecutor interfaces.Node + metrics *workflowMetrics + eventConfig *config.EventConfig + clusterID string + executionEnvClient pluginscore.ExecutionEnvClient // TODO @hamersaw - use to delete environments on success / failure } func (c *workflowExecutor) constructWorkflowMetadataPrefix(ctx context.Context, w *v1alpha1.FlyteWorkflow) (storage.DataReference, error) { @@ -504,7 +506,7 @@ func (c *workflowExecutor) cleanupRunningNodes(ctx context.Context, w v1alpha1.E func NewExecutor(ctx context.Context, store *storage.DataStore, enQWorkflow v1alpha1.EnqueueWorkflow, eventSink events.EventSink, k8sEventRecorder record.EventRecorder, metadataPrefix string, nodeExecutor interfaces.Node, eventConfig *config.EventConfig, - clusterID string, scope promutils.Scope) (executors.Workflow, error) { + clusterID string, executionEnvClient pluginscore.ExecutionEnvClient, scope promutils.Scope) (executors.Workflow, error) { basePrefix := store.GetBaseContainerFQN(ctx) if metadataPrefix != "" { var err error @@ -518,15 +520,16 @@ func NewExecutor(ctx context.Context, store *storage.DataStore, enQWorkflow v1al workflowScope := scope.NewSubScope("workflow") return &workflowExecutor{ - nodeExecutor: nodeExecutor, - store: store, - enqueueWorkflow: enQWorkflow, - wfRecorder: events.NewWorkflowEventRecorder(eventSink, workflowScope, store), - k8sRecorder: k8sEventRecorder, - metadataPrefix: basePrefix, - metrics: newMetrics(workflowScope), - eventConfig: eventConfig, - clusterID: clusterID, + nodeExecutor: nodeExecutor, + store: store, + enqueueWorkflow: enQWorkflow, + wfRecorder: events.NewWorkflowEventRecorder(eventSink, workflowScope, store), + k8sRecorder: k8sEventRecorder, + metadataPrefix: basePrefix, + metrics: newMetrics(workflowScope), + eventConfig: eventConfig, + clusterID: clusterID, + executionEnvClient: executionEnvClient, }, nil } diff --git a/flytepropeller/pkg/controller/workflow/executor_test.go b/flytepropeller/pkg/controller/workflow/executor_test.go index 807b71e1f4..8e061cb0de 100644 --- a/flytepropeller/pkg/controller/workflow/executor_test.go +++ b/flytepropeller/pkg/controller/workflow/executor_test.go @@ -25,6 +25,7 @@ import ( "github.com/flyteorg/flyte/flyteidl/gen/pb-go/flyteidl/event" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery" pluginCore "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core" + pluginsmocks "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/core/mocks" "github.com/flyteorg/flyte/flyteplugins/go/tasks/pluginmachinery/ioutils" "github.com/flyteorg/flyte/flytepropeller/events" eventsErr "github.com/flyteorg/flyte/flytepropeller/events/errors" @@ -51,10 +52,11 @@ import ( ) var ( - testScope = promutils.NewScope("test_wfexec") - fakeKubeClient = executorMocks.NewFakeKubeClient() - mockClientset = k8sfake.NewSimpleClientset() - signalClient = &gateMocks.SignalServiceClient{} + testScope = promutils.NewScope("test_wfexec") + fakeKubeClient = executorMocks.NewFakeKubeClient() + mockClientset = k8sfake.NewSimpleClientset() + signalClient = &gateMocks.SignalServiceClient{} + executionEnvClient = &pluginsmocks.ExecutionEnvClient{} ) const ( @@ -245,10 +247,11 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_Error(t *testing.T) { handlerFactory, err := factory.NewHandlerFactory(ctx, adminClient, adminClient, fakeKubeClient, mockClientset, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, promutils.NewTestScope()) assert.NoError(t, err) - nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, - maxOutputSize, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, promutils.NewTestScope()) + nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, maxOutputSize, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) assert.NoError(t, executor.Initialize(ctx)) @@ -328,11 +331,12 @@ func TestWorkflowExecutor_HandleFlyteWorkflow(t *testing.T) { handlerFactory, err := factory.NewHandlerFactory(ctx, adminClient, adminClient, fakeKubeClient, mockClientset, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, promutils.NewTestScope()) assert.NoError(t, err) - nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, - maxOutputSize, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, promutils.NewTestScope()) + nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, maxOutputSize, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) assert.NoError(t, executor.Initialize(ctx)) @@ -392,11 +396,12 @@ func BenchmarkWorkflowExecutor(b *testing.B) { recoveryClient := &recoveryMocks.Client{} adminClient := launchplan.NewFailFastLaunchPlanExecutor() handlerFactory := &nodemocks.HandlerFactory{} - nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, - maxOutputSize, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, scope) + nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, maxOutputSize, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, executionEnvClient, scope) assert.NoError(b, err) - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(b, err) assert.NoError(b, executor.Initialize(ctx)) @@ -504,10 +509,11 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_Failing(t *testing.T) { handlerFactory.OnSetupMatch(mock.Anything, mock.Anything, mock.Anything).Return(nil) handlerFactory.OnGetHandlerMatch(mock.Anything).Return(h, nil) - nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, - maxOutputSize, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, promutils.NewTestScope()) + nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, maxOutputSize, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) assert.NoError(t, executor.Initialize(ctx)) @@ -606,10 +612,11 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_Events(t *testing.T) { handlerFactory, err := factory.NewHandlerFactory(ctx, adminClient, adminClient, fakeKubeClient, mockClientset, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, promutils.NewTestScope()) assert.NoError(t, err) - nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, - maxOutputSize, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, promutils.NewTestScope()) + nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, eventSink, adminClient, adminClient, maxOutputSize, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) assert.NoError(t, executor.Initialize(ctx)) @@ -672,8 +679,8 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_EventFailure(t *testing.T) { handlerFactory := &nodemocks.HandlerFactory{} handlerFactory.OnSetupMatch(mock.Anything, mock.Anything, mock.Anything).Return(nil) handlerFactory.OnGetHandlerMatch(mock.Anything).Return(h, nil) - nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, nodeEventSink, adminClient, adminClient, - maxOutputSize, "s3://bucket", fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, promutils.NewTestScope()) + nodeExec, err := nodes.NewExecutor(ctx, config.GetConfig().NodeConfig, store, enqueueWorkflow, nodeEventSink, adminClient, adminClient, maxOutputSize, "s3://bucket", + fakeKubeClient, catalogClient, recoveryClient, eventConfig, testClusterID, signalClient, handlerFactory, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) t.Run("EventAlreadyInTerminalStateError", func(t *testing.T) { @@ -684,7 +691,8 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_EventFailure(t *testing.T) { Cause: errors.New("already exists"), } } - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) w := &v1alpha1.FlyteWorkflow{} assert.NoError(t, json.Unmarshal(wJSON, w)) @@ -703,7 +711,8 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_EventFailure(t *testing.T) { Cause: errors.New("already exists"), } } - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) w := &v1alpha1.FlyteWorkflow{} assert.NoError(t, json.Unmarshal(wJSON, w)) @@ -719,7 +728,8 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_EventFailure(t *testing.T) { Cause: errors.New("generic exists"), } } - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) w := &v1alpha1.FlyteWorkflow{} assert.NoError(t, json.Unmarshal(wJSON, w)) @@ -736,7 +746,8 @@ func TestWorkflowExecutor_HandleFlyteWorkflow_EventFailure(t *testing.T) { Cause: errors.New("incompatible cluster"), } } - executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", nodeExec, eventConfig, testClusterID, promutils.NewTestScope()) + executor, err := NewExecutor(ctx, store, enqueueWorkflow, eventSink, recorder, "metadata", + nodeExec, eventConfig, testClusterID, executionEnvClient, promutils.NewTestScope()) assert.NoError(t, err) w := &v1alpha1.FlyteWorkflow{} assert.NoError(t, json.Unmarshal(wJSON, w)) diff --git a/flytepropeller/plugins/loader.go b/flytepropeller/plugins/loader.go index a2983abb15..c38fd9675a 100644 --- a/flytepropeller/plugins/loader.go +++ b/flytepropeller/plugins/loader.go @@ -2,6 +2,9 @@ package plugins import ( + // Importing external plugins + _ "github.com/unionai/flyte/fasttask/plugin" + // Common place to import all plugins, so that it can be imported by Singlebinary (flytelite) or by propeller main _ "github.com/flyteorg/flyte/flyteplugins/go/tasks/plugins/array/awsbatch" _ "github.com/flyteorg/flyte/flyteplugins/go/tasks/plugins/array/k8s" diff --git a/go.mod b/go.mod index 4cf683c52a..41f63f51b8 100644 --- a/go.mod +++ b/go.mod @@ -182,6 +182,7 @@ require ( github.com/stretchr/objx v0.5.1 // indirect github.com/stretchr/testify v1.8.4 // indirect github.com/subosito/gotenv v1.6.0 // indirect + github.com/unionai/flyte/fasttask/plugin v0.0.0-00010101000000-000000000000 // indirect go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 // indirect go.opentelemetry.io/otel v1.24.0 // indirect @@ -250,6 +251,7 @@ replace ( github.com/flyteorg/flyte/flytestdlib => ./flytestdlib github.com/google/gnostic-models => github.com/google/gnostic-models v0.6.8 github.com/robfig/cron/v3 => github.com/unionai/cron/v3 v3.0.2-0.20220915080349-5790c370e63a + github.com/unionai/flyte/fasttask/plugin => ./fasttask/plugin k8s.io/api => k8s.io/api v0.28.2 k8s.io/apimachinery => k8s.io/apimachinery v0.28.2 k8s.io/client-go => k8s.io/client-go v0.28.2