diff --git a/.gitignore b/.gitignore index d59a1d7131..62b57bdcce 100644 --- a/.gitignore +++ b/.gitignore @@ -85,3 +85,6 @@ __pycache__ # Coverage .coverage .coverage* + +# kfp local execution default directory +local_outputs/ diff --git a/backend/src/apiserver/model/run.go b/backend/src/apiserver/model/run.go index b310bc00c3..26cc7c3685 100644 --- a/backend/src/apiserver/model/run.go +++ b/backend/src/apiserver/model/run.go @@ -363,11 +363,11 @@ var runAPIToModelFieldMap = map[string]string{ "storage_state": "StorageState", "status": "Conditions", "namespace": "Namespace", // v2beta1 API - "experiment_id": "ExperimentId", // v2beta1 API + "experiment_id": "ExperimentUUID", // v2beta1 API "state": "State", // v2beta1 API "state_history": "StateHistory", // v2beta1 API "runtime_details": "PipelineRuntimeManifest", // v2beta1 API - "recurring_run_id": "RecurringRunId", // v2beta1 API + "recurring_run_id": "JobUUID", // v2beta1 API } // APIToModelFieldMap returns a map from API names to field names for model Run. diff --git a/backend/src/apiserver/model/task.go b/backend/src/apiserver/model/task.go index 52fd3c8f6f..5c5d7092e6 100644 --- a/backend/src/apiserver/model/task.go +++ b/backend/src/apiserver/model/task.go @@ -81,8 +81,8 @@ var taskAPIToModelFieldMap = map[string]string{ "namespace": "Namespace", "pipeline_name": "PipelineName", // v2beta1 API "pipelineName": "PipelineName", // v1beta1 API - "run_id": "RunId", // v2beta1 API - "runId": "RunId", // v1beta1 API + "run_id": "RunUUID", // v2beta1 API + "runId": "RunUUID", // v1beta1 API "display_name": "Name", // v2beta1 API "execution_id": "MLMDExecutionID", // v2beta1 API "create_time": "CreatedTimestamp", // v2beta1 API @@ -91,7 +91,7 @@ var taskAPIToModelFieldMap = map[string]string{ "fingerprint": "Fingerprint", "state": "State", // v2beta1 API "state_history": "StateHistory", // v2beta1 API - "parent_task_id": "ParentTaskId", // v2beta1 API + "parent_task_id": "ParentTaskUUID", // v2beta1 API "mlmdExecutionID": "MLMDExecutionID", // v1beta1 API "created_at": "CreatedTimestamp", // v1beta1 API "finished_at": "FinishedTimestamp", // v1beta1 API diff --git a/backend/src/apiserver/resource/resource_manager.go b/backend/src/apiserver/resource/resource_manager.go index 91ff10cbe3..1ed779733f 100644 --- a/backend/src/apiserver/resource/resource_manager.go +++ b/backend/src/apiserver/resource/resource_manager.go @@ -1722,7 +1722,7 @@ func (r *ResourceManager) IsAuthorized(ctx context.Context, resourceAttributes * v1.CreateOptions{}, ) if err != nil { - if err, ok := err.(net.Error); ok && err.Timeout() { + if netError, ok := err.(net.Error); ok && netError.Timeout() { reportErr := util.NewUnavailableServerError( err, "Failed to create SubjectAccessReview for user '%s' (request: %+v) - try again later", diff --git a/backend/src/apiserver/server/pipeline_server.go b/backend/src/apiserver/server/pipeline_server.go index 0ef3f7c7ae..d3b2d0bd93 100644 --- a/backend/src/apiserver/server/pipeline_server.go +++ b/backend/src/apiserver/server/pipeline_server.go @@ -314,7 +314,7 @@ func (s *PipelineServer) getPipelineByName(ctx context.Context, name string, nam switch apiRequestVersion { case "v1beta1": return s.resourceManager.GetPipelineByNameAndNamespaceV1(name, namespace) - case "V2beta1": + case "v2beta1": p, err := s.resourceManager.GetPipelineByNameAndNamespace(name, namespace) return p, nil, err default: diff --git a/backend/src/apiserver/storage/default_experiment_store_test.go b/backend/src/apiserver/storage/default_experiment_store_test.go index e60f31da66..f0afe180f1 100644 --- a/backend/src/apiserver/storage/default_experiment_store_test.go +++ b/backend/src/apiserver/storage/default_experiment_store_test.go @@ -15,6 +15,7 @@ package storage import ( + "github.com/kubeflow/pipelines/backend/src/apiserver/model" "testing" "github.com/stretchr/testify/assert" @@ -93,3 +94,9 @@ func TestUnsetDefaultExperimentIdIfIdMatches(t *testing.T) { db.Close() } + +func TestExperimentAPIFieldMap(t *testing.T) { + for _, modelField := range (&model.Experiment{}).APIToModelFieldMap() { + assert.Contains(t, experimentColumns, modelField) + } +} diff --git a/backend/src/apiserver/storage/job_store_test.go b/backend/src/apiserver/storage/job_store_test.go index eb0ca34054..2dc86d4850 100644 --- a/backend/src/apiserver/storage/job_store_test.go +++ b/backend/src/apiserver/storage/job_store_test.go @@ -964,3 +964,9 @@ func TestDeleteJob_InternalError(t *testing.T) { assert.Equal(t, codes.Internal, err.(*util.UserError).ExternalStatusCode(), "Expected delete job to return internal error") } + +func TestJobAPIFieldMap(t *testing.T) { + for _, modelField := range (&model.Job{}).APIToModelFieldMap() { + assert.Contains(t, jobColumns, modelField) + } +} diff --git a/backend/src/apiserver/storage/run_store_test.go b/backend/src/apiserver/storage/run_store_test.go index ce6fcba0c0..fccbf7d3a9 100644 --- a/backend/src/apiserver/storage/run_store_test.go +++ b/backend/src/apiserver/storage/run_store_test.go @@ -1421,3 +1421,9 @@ func TestParseResourceReferences(t *testing.T) { assert.Nil(t, err) assert.Equal(t, expectedResourceReferences, actualResourceReferences) } + +func TestRunAPIFieldMap(t *testing.T) { + for _, modelField := range (&model.Run{}).APIToModelFieldMap() { + assert.Contains(t, runColumns, modelField) + } +} diff --git a/backend/src/apiserver/storage/task_store_test.go b/backend/src/apiserver/storage/task_store_test.go index 1d0e9cbff8..d2e495ce89 100644 --- a/backend/src/apiserver/storage/task_store_test.go +++ b/backend/src/apiserver/storage/task_store_test.go @@ -617,3 +617,9 @@ func TestTaskStore_UpdateOrCreateTasks(t *testing.T) { }) } } + +func TestTaskAPIFieldMap(t *testing.T) { + for _, modelField := range (&model.Task{}).APIToModelFieldMap() { + assert.Contains(t, taskColumns, modelField) + } +} diff --git a/backend/src/v2/compiler/argocompiler/argo_test.go b/backend/src/v2/compiler/argocompiler/argo_test.go index b5da4bf923..6c92e54574 100644 --- a/backend/src/v2/compiler/argocompiler/argo_test.go +++ b/backend/src/v2/compiler/argocompiler/argo_test.go @@ -52,6 +52,11 @@ func Test_argo_compiler(t *testing.T) { platformSpecPath: "../testdata/create_mount_delete_dynamic_pvc_platform.json", argoYAMLPath: "testdata/create_mount_delete_dynamic_pvc.yaml", }, + { + jobPath: "../testdata/hello_world.json", + platformSpecPath: "../testdata/create_pod_metadata.json", + argoYAMLPath: "testdata/create_pod_metadata.yaml", + }, } for _, tt := range tests { t.Run(fmt.Sprintf("%+v", tt), func(t *testing.T) { diff --git a/backend/src/v2/compiler/argocompiler/container.go b/backend/src/v2/compiler/argocompiler/container.go index 03d85f1801..f09241468a 100644 --- a/backend/src/v2/compiler/argocompiler/container.go +++ b/backend/src/v2/compiler/argocompiler/container.go @@ -15,19 +15,22 @@ package argocompiler import ( - wfapi "github.com/argoproj/argo-workflows/v3/pkg/apis/workflow/v1alpha1" "os" + + wfapi "github.com/argoproj/argo-workflows/v3/pkg/apis/workflow/v1alpha1" + "github.com/golang/protobuf/jsonpb" "github.com/kubeflow/pipelines/api/v2alpha1/go/pipelinespec" "github.com/kubeflow/pipelines/backend/src/v2/component" + "github.com/kubeflow/pipelines/kubernetes_platform/go/kubernetesplatform" k8score "k8s.io/api/core/v1" ) const ( volumeNameKFPLauncher = "kfp-launcher" - DefaultLauncherImage = "gcr.io/ml-pipeline/kfp-launcher@sha256:80cf120abd125db84fa547640fd6386c4b2a26936e0c2b04a7d3634991a850a4" + DefaultLauncherImage = "gcr.io/ml-pipeline/kfp-launcher@sha256:80cf120abd125db84fa547640fd6386c4b2a26936e0c2b04a7d3634991a850a4" LauncherImageEnvVar = "V2_LAUNCHER_IMAGE" - DefaultDriverImage = "gcr.io/ml-pipeline/kfp-driver@sha256:8e60086b04d92b657898a310ca9757631d58547e76bbbb8bfc376d654bef1707" - DriverImageEnvVar = "V2_DRIVER_IMAGE" + DefaultDriverImage = "gcr.io/ml-pipeline/kfp-driver@sha256:8e60086b04d92b657898a310ca9757631d58547e76bbbb8bfc376d654bef1707" + DriverImageEnvVar = "V2_DRIVER_IMAGE" ) func (c *workflowCompiler) Container(name string, component *pipelinespec.ComponentSpec, container *pipelinespec.PipelineDeploymentConfig_PipelineContainerSpec) error { @@ -58,19 +61,19 @@ type containerDriverInputs struct { } func GetLauncherImage() string { - launcherImage := os.Getenv(LauncherImageEnvVar) - if launcherImage == "" { - launcherImage = DefaultLauncherImage - } - return launcherImage + launcherImage := os.Getenv(LauncherImageEnvVar) + if launcherImage == "" { + launcherImage = DefaultLauncherImage + } + return launcherImage } func GetDriverImage() string { - driverImage := os.Getenv(DriverImageEnvVar) - if driverImage == "" { - driverImage = DefaultDriverImage - } - return driverImage + driverImage := os.Getenv(DriverImageEnvVar) + if driverImage == "" { + driverImage = DefaultDriverImage + } + return driverImage } func (c *workflowCompiler) containerDriverTask(name string, inputs containerDriverInputs) (*wfapi.DAGTask, *containerDriverOutputs) { @@ -169,14 +172,14 @@ type containerExecutorInputs struct { // name: argo workflows DAG task name // The other arguments are argo workflows task parameters, they can be either a // string or a placeholder. -func (c *workflowCompiler) containerExecutorTask(name string, inputs containerExecutorInputs) *wfapi.DAGTask { +func (c *workflowCompiler) containerExecutorTask(name string, inputs containerExecutorInputs, refName string) *wfapi.DAGTask { when := "" if inputs.condition != "" { when = inputs.condition + " != false" } return &wfapi.DAGTask{ Name: name, - Template: c.addContainerExecutorTemplate(), + Template: c.addContainerExecutorTemplate(refName), When: when, Arguments: wfapi.Arguments{ Parameters: []wfapi.Parameter{ @@ -191,7 +194,7 @@ func (c *workflowCompiler) containerExecutorTask(name string, inputs containerEx // any container component task. // During runtime, it's expected that pod-spec-patch will specify command, args // and resources etc, that are different for different tasks. -func (c *workflowCompiler) addContainerExecutorTemplate() string { +func (c *workflowCompiler) addContainerExecutorTemplate(refName string) string { // container template is parent of container implementation template nameContainerExecutor := "system-container-executor" nameContainerImpl := "system-container-impl" @@ -273,7 +276,56 @@ func (c *workflowCompiler) addContainerExecutorTemplate() string { Env: commonEnvs, }, } + // Update pod metadata if it defined in the Kubernetes Spec + if kubernetesConfigString, ok := c.wf.Annotations[annotationKubernetesSpec+refName]; ok { + k8sExecCfg := &kubernetesplatform.KubernetesExecutorConfig{} + if err := jsonpb.UnmarshalString(kubernetesConfigString, k8sExecCfg); err == nil { + extendPodMetadata(&executor.Metadata, k8sExecCfg) + } + } c.templates[nameContainerImpl] = executor c.wf.Spec.Templates = append(c.wf.Spec.Templates, *container, *executor) return nameContainerExecutor } + +// Extends the PodMetadata to include Kubernetes-specific executor config. +// Although the current podMetadata object is always empty, this function +// doesn't overwrite the existing podMetadata because for security reasons +// the existing podMetadata should have higher privilege than the user definition. +func extendPodMetadata( + podMetadata *wfapi.Metadata, + kubernetesExecutorConfig *kubernetesplatform.KubernetesExecutorConfig, +) { + // Get pod metadata information + if kubernetesExecutorConfig.GetPodMetadata() != nil { + labels := kubernetesExecutorConfig.GetPodMetadata().GetLabels() + if labels != nil { + if podMetadata.Labels == nil { + podMetadata.Labels = labels + } else { + podMetadata.Labels = extendMetadataMap(podMetadata.Labels, labels) + } + } + annotations := kubernetesExecutorConfig.GetPodMetadata().GetAnnotations() + if annotations != nil { + if podMetadata.Annotations == nil { + podMetadata.Annotations = annotations + } else { + podMetadata.Annotations = extendMetadataMap(podMetadata.Annotations, annotations) + } + } + } +} + +// Extends metadata map values, highPriorityMap should overwrites lowPriorityMap values +// The original Map inputs should have higher priority since its defined by admin +// TODO: Use maps.Copy after moving to go 1.21+ +func extendMetadataMap( + highPriorityMap map[string]string, + lowPriorityMap map[string]string, +) map[string]string { + for k, v := range highPriorityMap { + lowPriorityMap[k] = v + } + return lowPriorityMap +} diff --git a/backend/src/v2/compiler/argocompiler/container_test.go b/backend/src/v2/compiler/argocompiler/container_test.go new file mode 100644 index 0000000000..f242d87a18 --- /dev/null +++ b/backend/src/v2/compiler/argocompiler/container_test.go @@ -0,0 +1,90 @@ +// Copyright 2021-2024 The Kubeflow Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package argocompiler + +import ( + "testing" + + wfapi "github.com/argoproj/argo-workflows/v3/pkg/apis/workflow/v1alpha1" + "github.com/kubeflow/pipelines/kubernetes_platform/go/kubernetesplatform" + "github.com/stretchr/testify/assert" +) + +func Test_extendPodMetadata(t *testing.T) { + tests := []struct { + name string + podMetadata *wfapi.Metadata + kubernetesExecutorConfig *kubernetesplatform.KubernetesExecutorConfig + expected *wfapi.Metadata + }{ + { + "Valid - add pod labels and annotations", + &wfapi.Metadata{}, + &kubernetesplatform.KubernetesExecutorConfig{ + PodMetadata: &kubernetesplatform.PodMetadata{ + Annotations: map[string]string{ + "run_id": "123456", + }, + Labels: map[string]string{ + "kubeflow.com/kfp": "pipeline-node", + }, + }, + }, + &wfapi.Metadata{ + Annotations: map[string]string{ + "run_id": "123456", + }, + Labels: map[string]string{ + "kubeflow.com/kfp": "pipeline-node", + }, + }, + }, + { + "Valid - try overwrite default pod labels and annotations", + &wfapi.Metadata{ + Annotations: map[string]string{ + "run_id": "654321", + }, + Labels: map[string]string{ + "kubeflow.com/kfp": "default-node", + }, + }, + &kubernetesplatform.KubernetesExecutorConfig{ + PodMetadata: &kubernetesplatform.PodMetadata{ + Annotations: map[string]string{ + "run_id": "123456", + }, + Labels: map[string]string{ + "kubeflow.com/kfp": "pipeline-node", + }, + }, + }, + &wfapi.Metadata{ + Annotations: map[string]string{ + "run_id": "654321", + }, + Labels: map[string]string{ + "kubeflow.com/kfp": "default-node", + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + extendPodMetadata(tt.podMetadata, tt.kubernetesExecutorConfig) + assert.Equal(t, tt.expected, tt.podMetadata) + }) + } +} diff --git a/backend/src/v2/compiler/argocompiler/dag.go b/backend/src/v2/compiler/argocompiler/dag.go index b334c4beb5..719a166a9a 100644 --- a/backend/src/v2/compiler/argocompiler/dag.go +++ b/backend/src/v2/compiler/argocompiler/dag.go @@ -232,7 +232,7 @@ func (c *workflowCompiler) task(name string, task *pipelinespec.PipelineTaskSpec podSpecPatch: driverOutputs.podSpecPatch, cachedDecision: driverOutputs.cached, condition: driverOutputs.condition, - }) + }, task.GetComponentRef().GetName()) executor.Depends = depends([]string{driverTaskName}) return []wfapi.DAGTask{*driver, *executor}, nil case *pipelinespec.PipelineDeploymentConfig_ExecutorSpec_Importer: diff --git a/backend/src/v2/compiler/argocompiler/testdata/create_pod_metadata.yaml b/backend/src/v2/compiler/argocompiler/testdata/create_pod_metadata.yaml new file mode 100644 index 0000000000..8b623b87e6 --- /dev/null +++ b/backend/src/v2/compiler/argocompiler/testdata/create_pod_metadata.yaml @@ -0,0 +1,283 @@ +apiVersion: argoproj.io/v1alpha1 +kind: Workflow +metadata: + annotations: + pipelines.kubeflow.org/components-comp-hello-world: '{"executorLabel":"exec-hello-world","inputDefinitions":{"parameters":{"text":{"type":"STRING"}}}}' + pipelines.kubeflow.org/components-root: '{"dag":{"tasks":{"hello-world":{"cachingOptions":{"enableCache":true},"componentRef":{"name":"comp-hello-world"},"inputs":{"parameters":{"text":{"componentInputParameter":"text"}}},"taskInfo":{"name":"hello-world"}}}},"inputDefinitions":{"parameters":{"text":{"type":"STRING"}}}}' + pipelines.kubeflow.org/implementations-comp-hello-world: '{"args":["--text","{{$.inputs.parameters[''text'']}}"],"command":["sh","-ec","program_path=$(mktemp)\nprintf + \"%s\" \"$0\" \u003e \"$program_path\"\npython3 -u \"$program_path\" \"$@\"\n","def + hello_world(text):\n print(text)\n return text\n\nimport argparse\n_parser + = argparse.ArgumentParser(prog=''Hello world'', description='''')\n_parser.add_argument(\"--text\", + dest=\"text\", type=str, required=True, default=argparse.SUPPRESS)\n_parsed_args + = vars(_parser.parse_args())\n\n_outputs = hello_world(**_parsed_args)\n"],"image":"python:3.7"}' + pipelines.kubeflow.org/kubernetes-comp-hello-world: '{"podMetadata":{"annotations":{"experiment_id":"234567","run_id":"123456"},"labels":{"kubeflow.com/common":"test","kubeflow.com/kfp":"pipeline-node"}}}' + creationTimestamp: null + generateName: hello-world- +spec: + arguments: {} + entrypoint: entrypoint + podMetadata: + annotations: + pipelines.kubeflow.org/v2_component: "true" + labels: + pipelines.kubeflow.org/v2_component: "true" + serviceAccountName: pipeline-runner + templates: + - container: + args: + - --type + - CONTAINER + - --pipeline_name + - namespace/n1/pipeline/hello-world + - --run_id + - '{{workflow.uid}}' + - --dag_execution_id + - '{{inputs.parameters.parent-dag-id}}' + - --component + - '{{inputs.parameters.component}}' + - --task + - '{{inputs.parameters.task}}' + - --container + - '{{inputs.parameters.container}}' + - --iteration_index + - '{{inputs.parameters.iteration-index}}' + - --cached_decision_path + - '{{outputs.parameters.cached-decision.path}}' + - --pod_spec_patch_path + - '{{outputs.parameters.pod-spec-patch.path}}' + - --condition_path + - '{{outputs.parameters.condition.path}}' + - --kubernetes_config + - '{{inputs.parameters.kubernetes-config}}' + command: + - driver + image: gcr.io/ml-pipeline/kfp-driver + name: "" + resources: + limits: + cpu: 500m + memory: 512Mi + requests: + cpu: 100m + memory: 64Mi + inputs: + parameters: + - name: component + - name: task + - name: container + - name: parent-dag-id + - default: "-1" + name: iteration-index + - default: "" + name: kubernetes-config + metadata: {} + name: system-container-driver + outputs: + parameters: + - name: pod-spec-patch + valueFrom: + default: "" + path: /tmp/outputs/pod-spec-patch + - default: "false" + name: cached-decision + valueFrom: + default: "false" + path: /tmp/outputs/cached-decision + - name: condition + valueFrom: + default: "true" + path: /tmp/outputs/condition + - dag: + tasks: + - arguments: + parameters: + - name: pod-spec-patch + value: '{{inputs.parameters.pod-spec-patch}}' + name: executor + template: system-container-impl + when: '{{inputs.parameters.cached-decision}} != true' + inputs: + parameters: + - name: pod-spec-patch + - default: "false" + name: cached-decision + metadata: {} + name: system-container-executor + outputs: {} + - container: + command: + - should-be-overridden-during-runtime + env: + - name: KFP_POD_NAME + valueFrom: + fieldRef: + fieldPath: metadata.name + - name: KFP_POD_UID + valueFrom: + fieldRef: + fieldPath: metadata.uid + envFrom: + - configMapRef: + name: metadata-grpc-configmap + optional: true + image: gcr.io/ml-pipeline/should-be-overridden-during-runtime + name: "" + resources: {} + volumeMounts: + - mountPath: /kfp-launcher + name: kfp-launcher + initContainers: + - command: + - launcher-v2 + - --copy + - /kfp-launcher/launch + image: gcr.io/ml-pipeline/kfp-launcher + name: kfp-launcher + resources: + limits: + cpu: 500m + memory: 128Mi + requests: + cpu: 100m + volumeMounts: + - mountPath: /kfp-launcher + name: kfp-launcher + inputs: + parameters: + - name: pod-spec-patch + metadata: + annotations: + experiment_id: "234567" + run_id: "123456" + labels: + kubeflow.com/common: test + kubeflow.com/kfp: pipeline-node + name: system-container-impl + outputs: {} + podSpecPatch: '{{inputs.parameters.pod-spec-patch}}' + volumes: + - emptyDir: {} + name: kfp-launcher + - dag: + tasks: + - arguments: + parameters: + - name: component + value: '{{workflow.annotations.pipelines.kubeflow.org/components-comp-hello-world}}' + - name: task + value: '{"cachingOptions":{"enableCache":true},"componentRef":{"name":"comp-hello-world"},"inputs":{"parameters":{"text":{"componentInputParameter":"text"}}},"taskInfo":{"name":"hello-world"}}' + - name: container + value: '{{workflow.annotations.pipelines.kubeflow.org/implementations-comp-hello-world}}' + - name: parent-dag-id + value: '{{inputs.parameters.parent-dag-id}}' + - name: kubernetes-config + value: '{{workflow.annotations.pipelines.kubeflow.org/kubernetes-comp-hello-world}}' + name: hello-world-driver + template: system-container-driver + - arguments: + parameters: + - name: pod-spec-patch + value: '{{tasks.hello-world-driver.outputs.parameters.pod-spec-patch}}' + - default: "false" + name: cached-decision + value: '{{tasks.hello-world-driver.outputs.parameters.cached-decision}}' + depends: hello-world-driver.Succeeded + name: hello-world + template: system-container-executor + inputs: + parameters: + - name: parent-dag-id + metadata: {} + name: root + outputs: {} + - container: + args: + - --type + - '{{inputs.parameters.driver-type}}' + - --pipeline_name + - namespace/n1/pipeline/hello-world + - --run_id + - '{{workflow.uid}}' + - --dag_execution_id + - '{{inputs.parameters.parent-dag-id}}' + - --component + - '{{inputs.parameters.component}}' + - --task + - '{{inputs.parameters.task}}' + - --runtime_config + - '{{inputs.parameters.runtime-config}}' + - --iteration_index + - '{{inputs.parameters.iteration-index}}' + - --execution_id_path + - '{{outputs.parameters.execution-id.path}}' + - --iteration_count_path + - '{{outputs.parameters.iteration-count.path}}' + - --condition_path + - '{{outputs.parameters.condition.path}}' + command: + - driver + image: gcr.io/ml-pipeline/kfp-driver + name: "" + resources: + limits: + cpu: 500m + memory: 512Mi + requests: + cpu: 100m + memory: 64Mi + inputs: + parameters: + - name: component + - default: "" + name: runtime-config + - default: "" + name: task + - default: "0" + name: parent-dag-id + - default: "-1" + name: iteration-index + - default: DAG + name: driver-type + metadata: {} + name: system-dag-driver + outputs: + parameters: + - name: execution-id + valueFrom: + path: /tmp/outputs/execution-id + - name: iteration-count + valueFrom: + default: "0" + path: /tmp/outputs/iteration-count + - name: condition + valueFrom: + default: "true" + path: /tmp/outputs/condition + - dag: + tasks: + - arguments: + parameters: + - name: component + value: '{{workflow.annotations.pipelines.kubeflow.org/components-root}}' + - name: runtime-config + value: '{"parameters":{"text":{"stringValue":"hi there"}}}' + - name: driver-type + value: ROOT_DAG + name: root-driver + template: system-dag-driver + - arguments: + parameters: + - name: parent-dag-id + value: '{{tasks.root-driver.outputs.parameters.execution-id}}' + - name: condition + value: "" + depends: root-driver.Succeeded + name: root + template: root + inputs: {} + metadata: {} + name: entrypoint + outputs: {} +status: + finishedAt: null + startedAt: null diff --git a/backend/src/v2/driver/driver.go b/backend/src/v2/driver/driver.go index 1e67ab9c43..5add7239af 100644 --- a/backend/src/v2/driver/driver.go +++ b/backend/src/v2/driver/driver.go @@ -512,6 +512,11 @@ func extendPodSpecPatch( } } + // Get image pull secret information + for _, imagePullSecret := range kubernetesExecutorConfig.GetImagePullSecret() { + podSpec.ImagePullSecrets = append(podSpec.ImagePullSecrets, k8score.LocalObjectReference{Name: imagePullSecret.GetSecretName()}) + } + return nil } diff --git a/backend/src/v2/driver/driver_test.go b/backend/src/v2/driver/driver_test.go index ec8516fb34..ff950cda13 100644 --- a/backend/src/v2/driver/driver_test.go +++ b/backend/src/v2/driver/driver_test.go @@ -605,3 +605,69 @@ func Test_extendPodSpecPatch_Secret(t *testing.T) { }) } } + +func Test_extendPodSpecPatch_ImagePullSecrets(t *testing.T) { + tests := []struct { + name string + k8sExecCfg *kubernetesplatform.KubernetesExecutorConfig + expected *k8score.PodSpec + }{ + { + "Valid - SecretA and SecretB", + &kubernetesplatform.KubernetesExecutorConfig{ + ImagePullSecret: []*kubernetesplatform.ImagePullSecret{ + {SecretName: "SecretA"}, + {SecretName: "SecretB"}, + }, + }, + &k8score.PodSpec{ + Containers: []k8score.Container{ + { + Name: "main", + }, + }, + ImagePullSecrets: []k8score.LocalObjectReference{ + {Name: "SecretA"}, + {Name: "SecretB"}, + }, + }, + }, + { + "Valid - No ImagePullSecrets", + &kubernetesplatform.KubernetesExecutorConfig{ + ImagePullSecret: []*kubernetesplatform.ImagePullSecret{}, + }, + &k8score.PodSpec{ + Containers: []k8score.Container{ + { + Name: "main", + }, + }, + }, + }, + { + "Valid - empty", + &kubernetesplatform.KubernetesExecutorConfig{}, + &k8score.PodSpec{ + Containers: []k8score.Container{ + { + Name: "main", + }, + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := &k8score.PodSpec{Containers: []k8score.Container{ + { + Name: "main", + }, + }} + err := extendPodSpecPatch(got, tt.k8sExecCfg, nil, nil) + assert.Nil(t, err) + assert.NotNil(t, got) + assert.Equal(t, tt.expected, got) + }) + } +} diff --git a/backend/src/v2/metadata/client.go b/backend/src/v2/metadata/client.go index d6f37183a7..89b26b2fca 100644 --- a/backend/src/v2/metadata/client.go +++ b/backend/src/v2/metadata/client.go @@ -700,29 +700,42 @@ func (c *Client) GetExecutionsInDAG(ctx context.Context, dag *DAG, pipeline *Pip // Note, because MLMD does not have index on custom properties right now, we // take a pipeline run context to limit the number of executions the DB needs to // iterate through to find sub-executions. - res, err := c.svc.GetExecutionsByContext(ctx, &pb.GetExecutionsByContextRequest{ - ContextId: pipeline.pipelineRunCtx.Id, - Options: &pb.ListOperationOptions{ - FilterQuery: &parentDAGFilter, - }, - }) - if err != nil { - return nil, err - } - execs := res.GetExecutions() - for _, e := range execs { - execution := &Execution{execution: e} - taskName := execution.TaskName() - if taskName == "" { - return nil, fmt.Errorf("empty task name for execution ID: %v", execution.GetID()) + + nextPageToken := "" + for { + res, err := c.svc.GetExecutionsByContext(ctx, &pb.GetExecutionsByContextRequest{ + ContextId: pipeline.pipelineRunCtx.Id, + Options: &pb.ListOperationOptions{ + FilterQuery: &parentDAGFilter, + NextPageToken: &nextPageToken, + }, + }) + if err != nil { + return nil, err + } + + execs := res.GetExecutions() + for _, e := range execs { + execution := &Execution{execution: e} + taskName := execution.TaskName() + if taskName == "" { + return nil, fmt.Errorf("empty task name for execution ID: %v", execution.GetID()) + } + existing, ok := executionsMap[taskName] + if ok { + // TODO(Bobgy): to support retry, we need to handle multiple tasks with the same task name. + return nil, fmt.Errorf("two tasks have the same task name %q, id1=%v id2=%v", taskName, existing.GetID(), execution.GetID()) + } + executionsMap[taskName] = execution } - existing, ok := executionsMap[taskName] - if ok { - // TODO(Bobgy): to support retry, we need to handle multiple tasks with the same task name. - return nil, fmt.Errorf("two tasks have the same task name %q, id1=%v id2=%v", taskName, existing.GetID(), execution.GetID()) + + nextPageToken = res.GetNextPageToken() + + if nextPageToken == "" { + break } - executionsMap[taskName] = execution } + return executionsMap, nil } diff --git a/backend/third_party_licenses/apiserver.csv b/backend/third_party_licenses/apiserver.csv index 2b9aec08eb..6188972c0f 100644 --- a/backend/third_party_licenses/apiserver.csv +++ b/backend/third_party_licenses/apiserver.csv @@ -79,6 +79,7 @@ github.com/kubeflow/kfp-tekton/tekton-catalog/tekton-exithandler/pkg/apis/exitha github.com/kubeflow/kfp-tekton/tekton-catalog/tekton-kfptask/pkg/apis/kfptask,https://github.com/kubeflow/kfp-tekton/blob/a75d4b3711ff/tekton-catalog/tekton-kfptask/LICENSE,Apache-2.0 github.com/kubeflow/pipelines/api/v2alpha1/go,https://github.com/kubeflow/pipelines/blob/58ce09e07d03/api/LICENSE,Apache-2.0 github.com/kubeflow/pipelines/backend,https://github.com/kubeflow/pipelines/blob/HEAD/LICENSE,Apache-2.0 +github.com/kubeflow/pipelines/kubernetes_platform/go/kubernetesplatform,https://github.com/kubeflow/pipelines/blob/f51dc39614e4/kubernetes_platform/LICENSE,Apache-2.0 github.com/kubeflow/pipelines/third_party/ml-metadata/go/ml_metadata,https://github.com/kubeflow/pipelines/blob/e1f0c010f800/third_party/ml-metadata/LICENSE,Apache-2.0 github.com/lann/builder,https://github.com/lann/builder/blob/47ae307949d0/LICENSE,MIT github.com/lann/ps,https://github.com/lann/ps/blob/62de8c46ede0/LICENSE,MIT diff --git a/backend/third_party_licenses/driver.csv b/backend/third_party_licenses/driver.csv index 3444025b37..f674d9aea2 100644 --- a/backend/third_party_licenses/driver.csv +++ b/backend/third_party_licenses/driver.csv @@ -31,7 +31,7 @@ github.com/josharian/intern,https://github.com/josharian/intern/blob/v1.0.0/lice github.com/json-iterator/go,https://github.com/json-iterator/go/blob/v1.1.12/LICENSE,MIT github.com/kubeflow/pipelines/api/v2alpha1/go,https://github.com/kubeflow/pipelines/blob/58ce09e07d03/api/LICENSE,Apache-2.0 github.com/kubeflow/pipelines/backend,https://github.com/kubeflow/pipelines/blob/HEAD/LICENSE,Apache-2.0 -github.com/kubeflow/pipelines/kubernetes_platform/go/kubernetesplatform,https://github.com/kubeflow/pipelines/blob/aac4408237df/kubernetes_platform/LICENSE,Apache-2.0 +github.com/kubeflow/pipelines/kubernetes_platform/go/kubernetesplatform,https://github.com/kubeflow/pipelines/blob/f51dc39614e4/kubernetes_platform/LICENSE,Apache-2.0 github.com/kubeflow/pipelines/third_party/ml-metadata/go/ml_metadata,https://github.com/kubeflow/pipelines/blob/e1f0c010f800/third_party/ml-metadata/LICENSE,Apache-2.0 github.com/mailru/easyjson,https://github.com/mailru/easyjson/blob/v0.7.7/LICENSE,MIT github.com/modern-go/concurrent,https://github.com/modern-go/concurrent/blob/bacd9c7ef1dd/LICENSE,Apache-2.0 diff --git a/components/aws/sagemaker/tests/integration_tests/scripts/run_integration_tests b/components/aws/sagemaker/tests/integration_tests/scripts/run_integration_tests index a31576c4f0..49641c8ffc 100755 --- a/components/aws/sagemaker/tests/integration_tests/scripts/run_integration_tests +++ b/components/aws/sagemaker/tests/integration_tests/scripts/run_integration_tests @@ -38,11 +38,12 @@ ROBOMAKER_EXECUTION_ROLE_ARN=${ROBOMAKER_EXECUTION_ROLE_ARN:-""} SKIP_FSX_TESTS=${SKIP_FSX_TESTS:-"false"} -ACK_RELEASE_VERSION=${ACK_RELEASE_VERSION:-"v1.2.1"} +ACK_RELEASE_VERSION=${ACK_RELEASE_VERSION:-"1.2.4"} HELM_EXPERIMENTAL_OCI=1 SERVICE=sagemaker CHART_EXPORT_PATH=/tmp/chart CHART_REF=sagemaker-chart +TEST_EXIT_STATUS=1 while getopts ":n:r:s:" opt; do case $opt in @@ -88,10 +89,6 @@ fi function cleanup() { set +e - #push to metrics to cloudwatch - echo "Pushing Codebuild stats to Cloudwatch." - python ../../codebuild/scripts/push_stats_to_cloudwatch.py - cleanup_kfp # If installation fails before ack installation resources should be freed. if [[ -v ACK_K8S_NAMESPACE ]]; then @@ -101,6 +98,10 @@ function cleanup() { [ "${SKIP_KFP_OIDC_SETUP}" == "false" ] && delete_oidc_role + #push to metrics to cloudwatch + echo "Pushing Codebuild stats to Cloudwatch." + python ../../codebuild/scripts/push_stats_to_cloudwatch.py + if [[ "${SKIP_FSX_TESTS}" == "false" ]]; then delete_fsx_instance # Sleep in order for the security group to detach before attempting to delete it @@ -280,6 +281,10 @@ function cleanup_kfp() { # If this fails, deleting the nodegroup later will clean it up anyway kill -9 $MINIO_PID || true fi + if [[ $TEST_EXIT_STATUS -gt 0 ]]; then + kubectl delete -k "github.com/kubeflow/pipelines/manifests/kustomize/env/cert-manager/dev?ref=$KFP_VERSION&timeout=90s" + kubectl delete -k "github.com/kubeflow/pipelines/manifests/kustomize/env/cert-manager/cluster-scoped-resources?ref=$KFP_VERSION&timeout=90s" + fi } if [[ -z "${EKS_EXISTING_CLUSTER}" ]]; then @@ -332,4 +337,6 @@ fi DIR_THIS_FILE="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -cd $DIR_THIS_FILE/../ && python -m pytest "${pytest_args[@]}" --junitxml ./integration_tests.xml -n 9 +cd $DIR_THIS_FILE/../ +python -m pytest "${pytest_args[@]}" --junitxml ./integration_tests.xml -n 9 +TEST_EXIT_STATUS=$? diff --git a/components/google-cloud/Dockerfile b/components/google-cloud/Dockerfile index 9960664755..986b54b1e2 100644 --- a/components/google-cloud/Dockerfile +++ b/components/google-cloud/Dockerfile @@ -44,7 +44,7 @@ RUN pip3 install -U "fsspec>=0.7.4" "gcsfs>=0.6.0" "pandas<=1.3.5" "scikit-learn RUN pip3 install -U google-cloud-notebooks # Install main package -RUN pip3 install "git+https://github.com/kubeflow/pipelines.git@google-cloud-pipeline-components-2.8.0#egg=google-cloud-pipeline-components&subdirectory=components/google-cloud" +RUN pip3 install "git+https://github.com/kubeflow/pipelines.git@google-cloud-pipeline-components-2.9.0#egg=google-cloud-pipeline-components&subdirectory=components/google-cloud" # Note that components can override the container entry ponint. ENTRYPOINT ["python3","-m","google_cloud_pipeline_components.container.v1.aiplatform.remote_runner"] diff --git a/components/google-cloud/RELEASE.md b/components/google-cloud/RELEASE.md index d4a9a162b8..234754d6d6 100644 --- a/components/google-cloud/RELEASE.md +++ b/components/google-cloud/RELEASE.md @@ -1,7 +1,20 @@ ## Upcoming release +* Fix the missing output of pipeline remote runner. `AutoMLImageTrainingJobRunOp` now passes the model artifacts correctly to downstream components. +* Fix the metadata of Model Evaluation resource when row based metrics is disabled in `preview.model_evaluation.evaluation_llm_text_generation_pipeline`. +* Support `Jinja2>=3.1.2,<4`. + +## Release 2.9.0 * Use `large_model_reference` for `model_reference_name` when uploading models from `preview.llm.rlhf_pipeline` instead of hardcoding value as `text-bison@001`. * Disable caching when resolving model display names for RLHF-tuned models so a unique name is generated on each `preview.llm.rlhf_pipeline` run. * Upload the tuned adapter to Model Registry instead of model checkpoint from `preview.llm.rlhf_pipeline`. +* Fix the naming of AutoSxS's question answering task. "question_answer" -> "question_answering". +* Add Vertex model get component (`v1.model.ModelGetOp`). +* Migrate to Protobuf 4 (`protobuf>=4.21.1,<5`). Require `kfp>=2.6.0`. +* Support setting version aliases in (`v1.model.ModelUploadOp`). +* Only run `preview.llm.bulk_inference` pipeline after RLHF tuning for third-party models when `eval_dataset` is provided. +* Update LLM Evaluation Pipelines to use `text-bison@002` model by default. +* Apply latest GCPC image vulnerability resolutions (base OS and software updates). +* Add `preview.llm.rlaif_pipeline` that tunes large-language models from AI feedback. ## Release 2.8.0 * Release AutoSxS pipeline to preview. diff --git a/components/google-cloud/docs/source/versions.json b/components/google-cloud/docs/source/versions.json index a3f7f92a2c..c2db9b2756 100644 --- a/components/google-cloud/docs/source/versions.json +++ b/components/google-cloud/docs/source/versions.json @@ -1,4 +1,9 @@ [ + { + "version": "https://google-cloud-pipeline-components.readthedocs.io/en/google-cloud-pipeline-components-2.9.0", + "title": "2.9.0", + "aliases": [] + }, { "version": "https://google-cloud-pipeline-components.readthedocs.io/en/google-cloud-pipeline-components-2.8.0", "title": "2.8.0", diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/arbiter_preprocess.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/batch_prediction_pairwise.py similarity index 63% rename from components/google-cloud/google_cloud_pipeline_components/_implementation/llm/arbiter_preprocess.py rename to components/google-cloud/google_cloud_pipeline_components/_implementation/llm/batch_prediction_pairwise.py index 344e8fe6db..2b42075c48 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/arbiter_preprocess.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/batch_prediction_pairwise.py @@ -1,4 +1,4 @@ -# Copyright 2023 The Kubeflow Authors. All Rights Reserved. +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -11,10 +11,10 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -"""KFP Container component for preprocessing predictions for the Arbiter.""" +"""Component for running LLM Batch Prediction jobs side-by-side.""" import os -from typing import Dict, List +from typing import Any, Dict, List from google_cloud_pipeline_components import _placeholders from google_cloud_pipeline_components import utils as gcpc_utils @@ -24,94 +24,94 @@ def _resolve_image() -> str: """Determines the image URI to create a container from.""" - return ( - os.environ.get('AUTOSXS_IMAGE_OVERRIDE') - or utils.get_default_image_uri('autosxs')) + return os.environ.get( + 'AUTOSXS_IMAGE_OVERRIDE' + ) or utils.get_default_image_uri('autosxs') # pylint: disable=unused-argument,dangerous-default-value @dsl.container_component -def arbiter_preprocess( +def batch_prediction_pairwise( + display_name: str, evaluation_dataset: str, id_columns: List[str], - response_column_a: str, - response_column_b: str, task: str, - is_bp_output_a: bool, - is_bp_output_b: bool, autorater_prompt_parameters: Dict[str, Dict[str, str]], + response_column_a: str, + response_column_b: str, preprocessed_evaluation_dataset: dsl.Output[dsl.Dataset], # pylint: disable=unused-argument # pytype: disable=unsupported-operands preprocessed_evaluation_dataset_uri: dsl.OutputPath(str), # pylint: disable=unused-argument # pytype: disable=invalid-annotation gcp_resources: dsl.OutputPath(str), # pytype: disable=invalid-annotation - prediction_uris_a: str = '', - prediction_uris_b: str = '', + metadata: dsl.OutputPath(Dict[str, Any]), # pytype: disable=invalid-annotation + model_a: str = '', + model_b: str = '', model_a_prompt_parameters: Dict[str, Dict[str, str]] = {}, model_b_prompt_parameters: Dict[str, Dict[str, str]] = {}, + model_a_parameters: Dict[str, str] = {}, + model_b_parameters: Dict[str, str] = {}, human_preference_column: str = '', ) -> dsl.ContainerSpec: # pylint: disable=g-doc-args - """Preprocesses predictions tables for the AutoSxS Arbiter. + """Runs up to two LLM Batch Prediction jobs side-by-side. Args: + display_name: Display name for the batch prediction job. evaluation_dataset: GCS or BigQuery URIs representing a dataset of prompts and responses. id_columns: The columns which distinguish unique evaluation examples. - response_column_a: The column containing responses for model a. - response_column_b: The column containing responses for model a. task: Task to evaluate. - output_path: Path to write the path where preprocessed predictions are - stored. - is_bp_output_a: If True, the prediction URIs will be parsed as if they came - from Vertex Batch Prediction, where response_column_a represents a field - in the model output containing the response. If False, the expected format - will be a table containing all model_prompt_parameters and the - response_column. - is_bp_output_b: If True, the prediction URIs will be parsed as if they came - from Vertex Batch Prediction, where response_column_b represents a field - in the model output containing the response. If False, the expected format - will be a table containing all model_prompt_parameters and the - response_column. - prediction_uris: A list of GCS or BigQuery URIs representing a dataset of - prompts and responses for model a. - prediction_uris: A list of GCS or BigQuery URIs representing a dataset of - prompts and responses for model b. + autorater_prompt_parameters: Map of autorater prompt template parameters to + columns or templates. + response_column_a: The column containing responses for model a. + response_column_b: The column containing responses for model b. + model_a: A fully-qualified model resource name + (`projects/{project}/locations/{location}/models/{model}@{version}`) or + publisher model resource name (`publishers/{publisher}/models/{model}`). + This parameter is optional if Model A responses are specified. + model_b: A fully-qualified model resource name + (`projects/{project}/locations/{location}/models/{model}@{version}`) or + publisher model resource name (`publishers/{publisher}/models/{model}`). + This parameter is optional if Model B responses are specified. model_a_prompt_parameters: Map of model A prompt template parameters to columns or templates. model_b_prompt_parameters: Map of model B prompt template parameters to columns or templates. - autorater_prompt_parameters: Map of autorater prompt template parameters to - columns or templates. + model_a_parameters: The parameters that govern the predictions from model A, + such as temperature or maximum output tokens. + model_b_parameters: The parameters that govern the predictions from model B, + such as temperature or maximum output tokens. human_preference_column: The column containing ground truths. The default value is an empty string if not be provided by users. Returns: preprocessed_evaluation_dataset: Dataset of the table containing the inputs - expected by the Arbiter. + expected by the Arbiter. preprocessed_evaluation_dataset_uri: URI of the table containing the inputs - expected by the Arbiter. + expected by the Arbiter. gcp_resources: Tracker for GCP resources created by this component. + metadata_path: Path to write the object that stores computed metrics + metadata for the task preprocess component. """ return gcpc_utils.build_serverless_customjob_container_spec( project=_placeholders.PROJECT_ID_PLACEHOLDER, location=_placeholders.LOCATION_PLACEHOLDER, custom_job_payload=utils.build_payload( - display_name='arbiter_preprocess', + display_name='batch_prediction_pairwise', machine_type='n1-standard-4', image_uri=_resolve_image(), args=[ '--', # Used to mark the start of component flags. - 'arbiter_preprocess', + 'batch_prediction_sxs', + f'--display_name={display_name}', f'--evaluation_dataset={evaluation_dataset}', - f'--prediction_uris_a={prediction_uris_a}', - f'--prediction_uris_b={prediction_uris_b}', ( '--id_columns=' "{{$.inputs.parameters['id_columns'].json_escape[0]}}" ), - ( - '--autorater_prompt_parameters=' - "{{$.inputs.parameters['autorater_prompt_parameters']" - '.json_escape[0]}}' - ), + f'--task={task}', + f'--project={_placeholders.PROJECT_ID_PLACEHOLDER}', + f'--location={_placeholders.LOCATION_PLACEHOLDER}', + f'--model_a={model_a}', + f'--model_b={model_b}', ( '--model_a_prompt_parameters=' "{{$.inputs.parameters['model_a_prompt_parameters']" @@ -122,14 +122,26 @@ def arbiter_preprocess( "{{$.inputs.parameters['model_b_prompt_parameters']" '.json_escape[0]}}' ), + ( + '--autorater_prompt_parameters=' + "{{$.inputs.parameters['autorater_prompt_parameters']" + '.json_escape[0]}}' + ), f'--response_column_a={response_column_a}', f'--response_column_b={response_column_b}', + ( + '--model_a_parameters=' + "{{$.inputs.parameters['model_a_parameters'].json_escape[0]}}" + ), + ( + '--model_b_parameters=' + "{{$.inputs.parameters['model_b_parameters'].json_escape[0]}}" + ), f'--human_preference_column={human_preference_column}', - f'--task={task}', - f'--is_batch_prediction_output_a={is_bp_output_a}', - f'--is_batch_prediction_output_b={is_bp_output_b}', - f'--output_dir={dsl.PIPELINE_ROOT_PLACEHOLDER}', + f'--staging_dir={dsl.PIPELINE_ROOT_PLACEHOLDER}', f'--preprocessed_evaluation_dataset_uri={preprocessed_evaluation_dataset_uri}', + f'--metadata_path={metadata}', + f'--gcp_resources_path={gcp_resources}', '--executor_input={{$.json_escape[1]}}', ], ), diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/env.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/env.py index c7511d222c..d195ba06f7 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/env.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/env.py @@ -12,11 +12,18 @@ # See the License for the specific language governing permissions and # limitations under the License. """A collection of constants shared across components and pipelines.""" + import os +from google_cloud_pipeline_components._implementation.llm.generated import refined_image_versions + def get_private_image_tag() -> str: - return os.getenv('PRIVATE_IMAGE_TAG') or '20231213_0507_RC00' + return os.getenv('PRIVATE_IMAGE_TAG') or refined_image_versions.IMAGE_TAG + + +def get_autosxs_image_tag() -> str: + return os.getenv('PRIVATE_IMAGE_TAG') or refined_image_versions.IMAGE_TAG def get_use_test_machine_spec() -> bool: @@ -45,6 +52,7 @@ def get_use_test_machine_spec() -> bool: os.getenv('PRIVATE_IMAGE_NAME_PREFIX') or 'rlhf_' ) PRIVATE_IMAGE_TAG: str = get_private_image_tag() +AUTOSXS_IMAGE_TAG: str = get_autosxs_image_tag() # Dataset variables: TRAIN_SPLIT: str = 'train' diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/function_based.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/function_based.py index 9e4bae85ca..ae23c3fa78 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/function_based.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/function_based.py @@ -572,3 +572,31 @@ def get_uri(artifact: dsl.Input[dsl.Artifact], is_dir: bool = False) -> str: # @dsl.component(base_image=_image.GCPC_IMAGE_TAG, install_kfp_package=False) def get_empty_string() -> str: return '' + + +@dsl.component(base_image=_image.GCPC_IMAGE_TAG, install_kfp_package=False) +def validate_rlhf_inputs( + large_model_reference: str, + eval_dataset: Optional[str] = None, +) -> None: + """Checks user-provided arguments are valid for the RLHF pipeline.""" + models_that_support_bulk_inference = { + 't5-small', + 't5-large', + 't5-xl', + 't5-xxl', + 'llama-2-7b', + 'llama-2-7b-chat', + 'llama-2-13b', + 'llama-2-13b-chat', + } + if ( + eval_dataset + and large_model_reference not in models_that_support_bulk_inference + ): + raise ValueError( + f'eval_dataset not supported for {large_model_reference}. ' + 'Please set this value to None when tuning this model. ' + 'This model can be evaluated after tuning using Batch or Online ' + 'Prediction.' + ) diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/generated/__init__.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/generated/__init__.py new file mode 100644 index 0000000000..c0b27fe241 --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/generated/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/generated/refined_image_versions.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/generated/refined_image_versions.py new file mode 100644 index 0000000000..82d26db8ee --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/generated/refined_image_versions.py @@ -0,0 +1,20 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +"""Image tags to use for ReFINED images in GCPC. + +DO NOT EDIT - This file is generated, manual changes will be overridden. +""" + +IMAGE_TAG = '20240210_0207' diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/autosxs_metrics_computer.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/model_evaluation_text_generation_pairwise.py similarity index 81% rename from components/google-cloud/google_cloud_pipeline_components/_implementation/llm/autosxs_metrics_computer.py rename to components/google-cloud/google_cloud_pipeline_components/_implementation/llm/model_evaluation_text_generation_pairwise.py index ede9a816f9..94f41c24da 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/autosxs_metrics_computer.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/model_evaluation_text_generation_pairwise.py @@ -11,7 +11,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -"""KFP Container component for computing AutoSXS metrics.""" +"""KFP Container component for computing aggregate pairwise metrics.""" import os @@ -29,18 +29,18 @@ def _resolve_image() -> str: @dsl.container_component -def autosxs_metrics_computer( +def model_evaluation_text_generation_pairwise( judgments_dir: str, - has_human_preference: bool, autosxs_metrics: dsl.Output[dsl.Metrics], # pylint: disable=unused-argument # pytype: disable=unsupported-operands gcp_resources: dsl.OutputPath(str), # pytype: disable=invalid-annotation + human_preference_column: str = '', ) -> dsl.ContainerSpec: # pylint: disable=g-doc-args """Compute AutoSXS metrics using judgments outputs from Arbiter. Args: judgments_dir: Path where store the Judgments. - has_human_preference: Boolean value. True if users provided human preference - data, otherwise false. + human_preference_column: The column containing ground truths. The default + value is an empty string if not be provided by users. Returns: autosxs_metrics: Autosxs win rate metrics and human alignment metrics. @@ -48,17 +48,16 @@ def autosxs_metrics_computer( """ return gcpc_utils.build_serverless_customjob_container_spec( project=_placeholders.PROJECT_ID_PLACEHOLDER, - # Hardcode location to us-central1 for text-bison availability. - location='us-central1', + location=_placeholders.LOCATION_PLACEHOLDER, custom_job_payload=utils.build_payload( - display_name='autosxs_metrics_computer', + display_name='model_evaluation_text_generation_pairwise', machine_type='n1-standard-4', image_uri=_resolve_image(), args=[ '--', # Used to mark the start of component flags. 'autosxs_metrics', f'--judgments_dir={judgments_dir}', - f'--has_human_preference={has_human_preference}', + f'--human_preference_column={human_preference_column}', '--executor_input={{$.json_escape[1]}}', ], ), diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/autosxs_arbiter.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/online_evaluation_pairwise.py similarity index 92% rename from components/google-cloud/google_cloud_pipeline_components/_implementation/llm/autosxs_arbiter.py rename to components/google-cloud/google_cloud_pipeline_components/_implementation/llm/online_evaluation_pairwise.py index 6269b41307..19d02f27bb 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/autosxs_arbiter.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/online_evaluation_pairwise.py @@ -11,7 +11,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -"""KFP Container component that performs AutoSxS.""" +"""KFP Container component that performs row-level pairwise evaluation.""" import os from typing import Any, Dict, List @@ -35,7 +35,7 @@ def _get_prediction_endpoint_overrides() -> str: @dsl.container_component -def autosxs_arbiter( +def online_evaluation_pairwise( inference_output_uri: str, id_columns: List[str], task: str, @@ -57,8 +57,8 @@ def autosxs_arbiter( human_preference_column: Human preference column included in our inference output. task: Evaluation task in the form {task}@{version}. task can be one of - "summarization", "question_answer". Version is an integer with 3 digits or - "latest". Ex: summarization@001 or question_answer@latest. + "summarization", "question_answering". Version is an integer with 3 digits + or "latest". Ex: summarization@001 or question_answering@latest. judgments_format: The format to write judgments to. Can be either 'json' or 'bigquery'. bigquery_destination_prefix: BigQuery table to write judgments to if the @@ -74,10 +74,9 @@ def autosxs_arbiter( """ return gcpc_utils.build_serverless_customjob_container_spec( project=_placeholders.PROJECT_ID_PLACEHOLDER, - # Hardcode location to us-central1 for text-bison availability. - location='us-central1', + location=_placeholders.LOCATION_PLACEHOLDER, custom_job_payload=utils.build_payload( - display_name='autosxs_arbiter', + display_name='online_evaluation_pairwise', machine_type='n1-standard-4', image_uri=_resolve_image(), args=[ diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/preference_data_formatter.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/preference_data_formatter.py new file mode 100644 index 0000000000..993c57207e --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/preference_data_formatter.py @@ -0,0 +1,143 @@ +# Copyright 2023 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Utility function to format the preference data.""" + +from kfp import dsl + +from google_cloud_pipeline_components import _image + + +# pylint: disable=g-import-not-at-top +@dsl.component(base_image=_image.GCPC_IMAGE_TAG, install_kfp_package=False) +def format_preference_input_data( + model_a_inference_dir_uri: str, + model_b_inference_dir_uri: str, + instruction: str, +) -> str: + """Format the inference data from model a and model b and merge them as the input for auto sxs evaluation. + + Args: + model_a_inference_dir_uri: Where the model a judgments data was saved in the + previous step. + model_b_inference_dir_uri: Where the model b judgments data was saved in the + previous step. + instruction: instruction to the task. + + Returns: + The path to the new output file that saved the formatted input data for + AutoSxs arbiter. + """ + import json + import hashlib + import os + import re + import glob + + model_a_inference_dir_uri = re.sub( + '^gs://', '/gcs/', model_a_inference_dir_uri + ) + model_b_inference_dir_uri = re.sub( + '^gs://', '/gcs/', model_b_inference_dir_uri + ) + + model_a_inference_data_map = {} + model_b_inference_data_map = {} + files_in_folder_a = glob.glob( + os.path.join(model_a_inference_dir_uri, 'text*') + ) + files_in_folder_b = glob.glob( + os.path.join(model_b_inference_dir_uri, 'text*') + ) + assert ( + len(files_in_folder_a) == 1 & len(files_in_folder_b) == 1 + ), 'There should be one inference data file for each model' + with open(files_in_folder_a[0], 'r') as inputs: + for line in inputs: + line_json = json.loads(line) + hash_obj = hashlib.md5( + json.dumps(line_json['inputs']['inputs_pretokenized']).encode() + ) + hash_int = int(hash_obj.hexdigest(), 16) + model_a_inference_data_map[str(hash_int)] = line_json + + with open(files_in_folder_b[0], 'r') as inputs: + for line in inputs: + line_json = json.loads(line) + hash_obj = hashlib.md5( + json.dumps(line_json['inputs']['inputs_pretokenized']).encode() + ) + hash_int = int(hash_obj.hexdigest(), 16) + model_b_inference_data_map[str(hash_int)] = line_json + + formatted_data_json = [] + for key, model_a_inference_item in model_a_inference_data_map.items(): + if key in model_b_inference_data_map: + model_b_inference_item = model_b_inference_data_map[key] + updated_line_json = {} + updated_line_json['inference_instruction'] = instruction + updated_line_json['content'] = model_a_inference_item['inputs'][ + 'inputs_pretokenized' + ] + updated_line_json['inference_context'] = model_a_inference_item['inputs'][ + 'inputs_pretokenized' + ] + updated_line_json['response_a'] = model_a_inference_item['prediction'] + updated_line_json['response_b'] = model_b_inference_item['prediction'] + formatted_data_json.append(updated_line_json) + + output_uri = files_in_folder_a[0].replace( + '.jsonl', '_formatted_for_autosxs.jsonl' + ) + with open(output_uri, 'w') as f: + for line in formatted_data_json: + f.write(json.dumps(line)) + f.write('\n') + return output_uri + + +# pylint: disable=g-import-not-at-top +@dsl.component(base_image=_image.GCPC_IMAGE_TAG, install_kfp_package=False) +def format_preference_data(input_uri: str) -> str: + """Format the input for preference data. + + Args: + input_uri: Where the judgments data was saved in the previous step. + + Returns: + The path to the new output file that saved the formatted preference data. + It's under the same folder as the original data file. + """ + import json + import re + + input_uri = re.sub('^gs://', '/gcs/', input_uri) + output_uri = input_uri.replace('.jsonl', '_formatted_for_rlaif.jsonl') + formatted_data_json = [] + with open(input_uri, 'r') as inputs: + for line in inputs: + line_json = json.loads(line) + if line_json['choice'] not in ['A', 'B']: + continue + updated_line_json = {} + updated_line_json['input_text'] = line_json['content'] + updated_line_json['candidate_0'] = line_json['response_a'] + updated_line_json['candidate_1'] = line_json['response_b'] + updated_line_json['choice'] = 0 if line_json['choice'] == 'A' else 1 + formatted_data_json.append(updated_line_json) + + with open(output_uri, 'w') as f: + for line in formatted_data_json: + f.write(json.dumps(line)) + f.write('\n') + return output_uri diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/task_preprocess.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/task_preprocess.py deleted file mode 100644 index c04b6aff5a..0000000000 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/task_preprocess.py +++ /dev/null @@ -1,97 +0,0 @@ -# Copyright 2023 The Kubeflow Authors. All Rights Reserved. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -"""Component for preprocessing the evaluation dataset into prediction inputs.""" - -import os -from typing import Any, Dict, List - -from google_cloud_pipeline_components import _placeholders -from google_cloud_pipeline_components import utils as gcpc_utils -from google_cloud_pipeline_components._implementation.llm import utils -from kfp import dsl - - -def _resolve_image() -> str: - """Determines the image URI to create a container from.""" - return ( - os.environ.get('AUTOSXS_IMAGE_OVERRIDE') - or utils.get_default_image_uri('autosxs')) - - -# pylint: disable=dangerous-default-value,g-bare-generic,unused-argument -@dsl.container_component -def task_preprocess( - evaluation_dataset: str, - id_columns: List[str], - task: str, - model_prompt_parameters: Dict[str, Dict[str, str]], - prediction_inputs: dsl.OutputPath(List[str]), # pytype: disable=invalid-annotation - gcp_resources: dsl.OutputPath(str), # pytype: disable=invalid-annotation - metadata: dsl.OutputPath(Dict[str, Any]), # pytype: disable=invalid-annotation - response_column: str, - human_preference_column: str = '', -) -> dsl.ContainerSpec: # pylint: disable=g-doc-args - """Preprocesses evaluation dataset into prediction inputs. - - Args: - evaluation_dataset: GCS or BigQuery URIs representing a dataset of prompts - and responses. - id_columns: The columns which distinguish unique evaluation examples. - task: Evaluation task in the form {task}@{version}. task can be one of - "summarization", "question_answer". Version is an integer with 3 digits or - "latest". Ex: summarization@001 or question_answer@latest. - model_prompt_parameters: Map of model prompt template parameters to columns - or templates. - response_column: Either an existing column containing predefined responses, - or the name of the model output column containing responses. - human_preference_column: The column containing ground truths. Only required - when users want to check the autorater alignment against human preference. - - Returns: - prediction_inputs_path: Path to write the path where preprocessed - predictions are stored. - gcp_resources: Tracker for GCP resources created by this component. - metadata_path: Path to write the object that stores computed metrics - metadata for the task preprocess component. - """ - return gcpc_utils.build_serverless_customjob_container_spec( - project=_placeholders.PROJECT_ID_PLACEHOLDER, - location=_placeholders.LOCATION_PLACEHOLDER, - custom_job_payload=utils.build_payload( - display_name='task_preprocess', - machine_type='n1-standard-4', - image_uri=_resolve_image(), - args=[ - '--', # Used to mark the start of component flags. - 'task_preprocess', - f'--evaluation_dataset={evaluation_dataset}', - f'--staging_dir={dsl.PIPELINE_ROOT_PLACEHOLDER}', - f'--task={task}', - f'--prediction_inputs_path={prediction_inputs}', - ( - '--id_columns=' - "{{$.inputs.parameters['id_columns'].json_escape[0]}}" - ), - ( - '--model_prompt_parameters=' - "{{$.inputs.parameters['model_prompt_parameters']" - '.json_escape[0]}}' - ), - f'--metadata_path={metadata}', - f'--response_column={response_column}', - f'--human_preference_column={human_preference_column}', - ], - ), - gcp_resources=gcp_resources, - ) diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils.py index 2c5a6369bc..e01bc5d9e6 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils.py @@ -13,7 +13,7 @@ # limitations under the License. """Utility functions used to create custom Kubeflow components.""" import os -from typing import Any, Dict, List +from typing import Any, Dict, List, Optional from google_cloud_pipeline_components._implementation.llm import env import kfp @@ -28,6 +28,8 @@ def build_payload( accelerator_type: str = '', accelerator_count: int = 0, encryption_spec_key_name: str = '', + labels: Optional[Dict[str, str]] = None, + scheduling: Optional[Dict[str, Any]] = None, ) -> Dict[str, Any]: """Generates payload for a custom training job. @@ -46,6 +48,8 @@ def build_payload( then all resources created by the CustomJob will be encrypted with the provided encryption key. Note that this is not supported for TPU at the moment. + labels: The labels with user-defined metadata to organize CustomJobs. + scheduling: Scheduling options for a CustomJob. Returns: Custom job payload. @@ -86,6 +90,12 @@ def build_payload( if encryption_spec_key_name: payload['encryption_spec'] = {'kms_key_name': encryption_spec_key_name} + if labels: + payload['labels'] = labels + + if scheduling: + payload['job_spec']['scheduling'] = scheduling + return payload @@ -113,9 +123,14 @@ def get_default_image_uri(image_name: str) -> str: Returns: URI of the image. """ + if image_name.find('autosxs') != -1: + image_tag = env.get_autosxs_image_tag() + else: + image_tag = env.get_private_image_tag() + return '/'.join([ f'{env.PRIVATE_ARTIFACT_REGISTRY_LOCATION}-docker.pkg.dev', env.PRIVATE_ARTIFACT_REGISTRY_PROJECT, env.PRIVATE_ARTIFACT_REGISTRY, - f'{env.PRIVATE_IMAGE_NAME_PREFIX}{image_name}:{env.get_private_image_tag()}', + f'{env.PRIVATE_IMAGE_NAME_PREFIX}{image_name}:{image_tag}', ]) diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils_test.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils_test.py index a16c878fb3..601522a845 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils_test.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/llm/utils_test.py @@ -57,6 +57,36 @@ def test_build_payload_with_encryption_spec_key_name(self): ) self.assertDictEqual(expected_payload, actual_payload) + def test_build_payload_with_labels_and_scheduling(self): + machine_type = "n1-standard-1" + image_uri = "fake_image_uri" + args = ["--foo=bar"] + labels = {"vertex-internal-enable-custom-job-retries": ""} + scheduling = {"disable_retries": False} + + expected_payload = { + "display_name": "test_with_encryption_spec_key_name", + "job_spec": { + "worker_pool_specs": [{ + "replica_count": "1", + "machine_spec": {"machine_type": machine_type}, + "container_spec": {"image_uri": image_uri, "args": args}, + }], + "scheduling": scheduling, + }, + "labels": labels, + } + + actual_payload = utils.build_payload( + display_name="test_with_encryption_spec_key_name", + machine_type=machine_type, + image_uri=image_uri, + args=args, + labels=labels, + scheduling=scheduling, + ) + self.assertDictEqual(expected_payload, actual_payload) + if __name__ == "__main__": unittest.main() diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/endpoint_batch_predict/component.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/endpoint_batch_predict/component.py index 5eb8373591..edf7070fdc 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/endpoint_batch_predict/component.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/endpoint_batch_predict/component.py @@ -16,6 +16,7 @@ from typing import Dict, List, NamedTuple, Optional, Union from google_cloud_pipeline_components import utils as gcpc_utils from google_cloud_pipeline_components._implementation.model_evaluation import utils +from google_cloud_pipeline_components._implementation.model_evaluation import version from kfp import dsl from kfp.dsl import Artifact from kfp.dsl import container_component @@ -26,7 +27,7 @@ _IMAGE_URI = 'us-docker.pkg.dev/vertex-evaluation/public/llm:wjess-fishfooding' -@dsl.component +@dsl.component(base_image=version.LLM_EVAL_IMAGE_TAG) def add_json_escape_parameters(parameters: dict) -> str: if not parameters: return @@ -36,7 +37,7 @@ def add_json_escape_parameters(parameters: dict) -> str: return json_escaped_parameters -@dsl.component +@dsl.component(base_image=version.LLM_EVAL_IMAGE_TAG) def add_json_escape_paths(paths: list) -> str: if not paths: return diff --git a/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/llm_evaluation_preprocessor/component.py b/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/llm_evaluation_preprocessor/component.py index ec31c28c9a..3468d0e28f 100644 --- a/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/llm_evaluation_preprocessor/component.py +++ b/components/google-cloud/google_cloud_pipeline_components/_implementation/model_evaluation/llm_evaluation_preprocessor/component.py @@ -22,7 +22,7 @@ # pylint: disable=g-import-not-at-top, g-doc-args, unexpected-keyword-arg -@dsl.component +@dsl.component(base_image=version.LLM_EVAL_IMAGE_TAG) def add_json_escape_to_list(input_list: List[str]) -> str: import json diff --git a/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/launcher.py b/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/launcher.py index 1662994efa..28d0f1a045 100644 --- a/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/launcher.py +++ b/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/launcher.py @@ -28,7 +28,15 @@ def _parse_args(args: List[str]): args.append('--payload') args.append('"{}"') # Unused but required by parser_util. parser, _ = parser_util.parse_default_args(args) - # Parse the conditionally required arguments + # Parse the conditionally required arguments. + parser.add_argument( + '--executor_input', + dest='executor_input', + type=str, + # executor_input is only needed for components that emit output artifacts. + required=True, + default=argparse.SUPPRESS, + ) parser.add_argument( '--display_name', dest='display_name', diff --git a/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/remote_runner.py b/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/remote_runner.py index a48616cc23..c46b8d3c39 100644 --- a/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/remote_runner.py +++ b/components/google-cloud/google_cloud_pipeline_components/container/v1/automl_training_job/image/remote_runner.py @@ -13,6 +13,7 @@ # limitations under the License. """GCP remote runner for AutoML image training pipelines based on the AI Platform SDK.""" +import json import logging from typing import Any, Dict, Optional, Sequence @@ -25,6 +26,7 @@ from google.cloud.aiplatform import training_jobs from google.cloud.aiplatform_v1.types import model from google.cloud.aiplatform_v1.types import training_pipeline +from google_cloud_pipeline_components.container.v1.aiplatform import remote_runner from google_cloud_pipeline_components.container.v1.gcp_launcher import pipeline_remote_runner from google_cloud_pipeline_components.container.v1.gcp_launcher.utils import error_util @@ -195,6 +197,7 @@ def create_pipeline( project: str, location: str, gcp_resources: str, + executor_input: str, **kwargs: Dict[str, Any], ): """Create and poll AutoML Vision training pipeline status till it reaches a final state. @@ -222,29 +225,39 @@ def create_pipeline( project: Project name. location: Location to start the training job. gcp_resources: URI for storing GCP resources. + executor_input: Pipeline executor input. **kwargs: Extra args for creating the payload. """ - remote_runner = pipeline_remote_runner.PipelineRemoteRunner( + runner = pipeline_remote_runner.PipelineRemoteRunner( type, project, location, gcp_resources ) try: # Create AutoML vision training pipeline if it does not exist - pipeline_name = remote_runner.check_if_pipeline_exists() + pipeline_name = runner.check_if_pipeline_exists() if pipeline_name is None: payload = create_payload(project, location, **kwargs) logging.info( 'AutoML Vision training payload formatted: %s', payload, ) - pipeline_name = remote_runner.create_pipeline( + pipeline_name = runner.create_pipeline( create_pipeline_with_client, payload, ) # Poll AutoML Vision training pipeline status until # "PipelineState.PIPELINE_STATE_SUCCEEDED" - remote_runner.poll_pipeline(get_pipeline_with_client, pipeline_name) + pipeline = runner.poll_pipeline(get_pipeline_with_client, pipeline_name) except (ConnectionError, RuntimeError) as err: error_util.exit_with_internal_error(err.args[0]) + return # No-op, suppressing uninitialized `pipeline` variable lint error. + + # Writes artifact output on success. + if not isinstance(pipeline, training_pipeline.TrainingPipeline): + raise ValueError('Internal error: no training pipeline was created.') + remote_runner.write_to_artifact( + json.loads(executor_input), + pipeline.model_to_upload.name, + ) diff --git a/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/__init__.py b/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/__init__.py new file mode 100644 index 0000000000..fd34574091 --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Google Cloud Pipeline Components - Get Model Launcher and Remote Runner.""" diff --git a/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/launcher.py b/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/launcher.py new file mode 100644 index 0000000000..675019d061 --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/launcher.py @@ -0,0 +1,59 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""GCP launcher for Get Model based on the AI Platform SDK.""" + +import argparse +import sys + +from google_cloud_pipeline_components.container.v1.model.get_model import remote_runner + + +def _parse_args(args): + """Parse command line arguments.""" + parser = argparse.ArgumentParser( + prog='Vertex Pipelines get model launcher', description='' + ) + parser.add_argument( + '--executor_input', + dest='executor_input', + type=str, + # executor_input is only needed for components that emit output artifacts. + required=True, + default=argparse.SUPPRESS, + ) + parser.add_argument('--project', dest='project', type=str) + parser.add_argument('--location', dest='location', type=str) + parser.add_argument('--model_name', dest='model_name', type=str) + parsed_args, _ = parser.parse_known_args(args) + return vars(parsed_args) + + +def main(argv): + """Main entry. + + Expected input args are as follows: + model_name - Required. Provided string resource name to create a model + artifact. + project - Required. Project to get this Model from. + location - Required. Location to get this Model from. + + Args: + argv: A list of system arguments. + """ + parsed_args = _parse_args(argv) + remote_runner.get_model(**parsed_args) + + +if __name__ == '__main__': + main(sys.argv[1:]) diff --git a/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/remote_runner.py b/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/remote_runner.py new file mode 100644 index 0000000000..797f8c6f53 --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/container/v1/model/get_model/remote_runner.py @@ -0,0 +1,52 @@ +# Copyright 2021 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Remote runner for Get Model based on the Vertex AI SDK.""" + +from google.api_core.client_options import ClientOptions +from google.cloud import aiplatform_v1 as aip_v1 +from google_cloud_pipeline_components.container.utils import artifact_utils +from google_cloud_pipeline_components.types import artifact_types + + +def get_model( + executor_input, + model_name: str, + project: str, + location: str, +) -> None: + """Get model.""" + if not location or not project: + raise ValueError( + 'Model resource name must be in the format' + ' projects/{project}/locations/{location}/models/{model_name}' + ) + api_endpoint = location + '-aiplatform.googleapis.com' + vertex_uri_prefix = f'https://{api_endpoint}/v1/' + model_resource_name = ( + f'projects/{project}/locations/{location}/models/{model_name}' + ) + + client_options = ClientOptions(api_endpoint=api_endpoint) + client = aip_v1.ModelServiceClient(client_options=client_options) + request = aip_v1.GetModelRequest(name=model_resource_name) + get_model_response = client.get_model(request) + resp_model_name_without_version = get_model_response.name.split('@', 1)[0] + model_resource_name = ( + f'{resp_model_name_without_version}@{get_model_response.version_id}' + ) + + vertex_model = artifact_types.VertexModel.create( + 'model', vertex_uri_prefix + model_resource_name, model_resource_name + ) + artifact_utils.update_output_artifacts(executor_input, [vertex_model]) diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_ensemble.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_ensemble.py index 876797edd4..340e64778d 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_ensemble.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_ensemble.py @@ -72,7 +72,7 @@ def automl_forecasting_ensemble( # fmt: on job_id = dsl.PIPELINE_JOB_ID_PLACEHOLDER task_id = dsl.PIPELINE_TASK_ID_PLACEHOLDER - image_uri = 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125' + image_uri = 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125' display_name = f'automl-forecasting-ensemble-{job_id}-{task_id}' error_file_path = f'{root_dir}/{job_id}/{task_id}/error.pb' diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_1_tuner.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_1_tuner.py index 3bc7d4eee2..d33f427977 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_1_tuner.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_1_tuner.py @@ -99,14 +99,14 @@ def automl_forecasting_stage_1_tuner( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125', '", "args": ["forecasting_mp_l2l_stage_1_tuner', '", "--region=', location, '", "--transform_output_path=', transform_output.uri, '", "--training_docker_uri=', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125', '", "--reduce_search_space_mode=', reduce_search_space_mode, f'", "--component_id={dsl.PIPELINE_TASK_ID_PLACEHOLDER}', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_2_tuner.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_2_tuner.py index 212e248ee8..577bc9a42d 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_2_tuner.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/forecasting_stage_2_tuner.py @@ -97,14 +97,14 @@ def automl_forecasting_stage_2_tuner( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125', '", "args": ["forecasting_mp_l2l_stage_2_tuner', '", "--region=', location, '", "--transform_output_path=', transform_output.uri, '", "--training_docker_uri=', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125', f'", "--component_id={dsl.PIPELINE_TASK_ID_PLACEHOLDER}', '", "--training_base_dir=', root_dir, diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/learn_to_learn_forecasting_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/learn_to_learn_forecasting_pipeline.yaml index adfe76bd6f..c91370d4e8 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/learn_to_learn_forecasting_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/learn_to_learn_forecasting_pipeline.yaml @@ -78,16 +78,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -102,9 +99,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -115,11 +110,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -159,10 +152,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-ensemble-2: executorLabel: exec-automl-forecasting-ensemble-2 @@ -172,16 +163,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -196,9 +184,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -209,11 +195,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -253,10 +237,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-1-tuner: executorLabel: exec-automl-forecasting-stage-1-tuner @@ -271,9 +253,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -286,9 +266,7 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the hyperparameter tuning should - - run.' + description: Number of hours the hyperparameter tuning should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -302,18 +280,16 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run hyperparameter tuning. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -324,22 +300,14 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "activation","categorical_value_spec": {"values": - - ["tanh"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "activation","categorical_value_spec": + {"values": ["tanh"]}}]' isOptional: true parameterType: LIST worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -351,11 +319,8 @@ components: description: The trained model and architectures. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-2-tuner: executorLabel: exec-automl-forecasting-stage-2-tuner @@ -370,16 +335,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The forecasting example gen - - metadata.' + description: The forecasting example gen metadata. transform_output: artifactType: schemaTitle: system.Artifact @@ -389,14 +350,11 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Path to the json of hyperparameter - - tuning results to use when evaluating models.' + description: Path to the json of hyperparameter tuning results to use when + evaluating models. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -410,9 +368,8 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model.' + description: Number of selected trials. The number of weak learners in the + final model. parameterType: NUMBER_INTEGER project: description: Project to run stage 2 tuner. @@ -425,13 +382,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -443,11 +395,8 @@ components: description: The trained (private) model artifact paths and their hyperparameters. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-finalizer: executorLabel: exec-automl-tabular-finalizer @@ -470,10 +419,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-calculate-training-parameters: executorLabel: exec-calculate-training-parameters @@ -971,6 +918,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -1689,6 +1639,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -2745,159 +2698,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -2907,24 +2826,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -2940,9 +2857,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -2957,17 +2872,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -3017,18 +2926,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -3038,9 +2942,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -3073,67 +2975,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -3143,9 +3024,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -3156,25 +3036,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -3198,264 +3073,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -3489,11 +3312,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -3514,36 +3335,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-finalize-eval-quantile-parameters: executorLabel: exec-finalize-eval-quantile-parameters @@ -5561,16 +5374,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -5578,9 +5387,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -5590,19 +5397,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -5617,76 +5417,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -5696,33 +5481,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -5732,45 +5508,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -5806,7 +5573,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5840,7 +5607,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5875,11 +5642,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_1_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--reduce_search_space_mode=", "{{$.inputs.parameters[''reduce_search_space_mode'']}}", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", @@ -5918,11 +5685,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_2_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", "\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -5961,7 +5728,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -6285,8 +6052,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -6303,7 +6070,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -6473,10 +6240,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6509,10 +6276,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6545,7 +6312,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-predictions-column-2: container: args: @@ -6574,7 +6341,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-importer: importer: artifactUri: @@ -7020,7 +6787,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-2: container: args: @@ -7049,7 +6816,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-set-optional-inputs: container: args: @@ -7112,7 +6879,7 @@ deploymentSpec: \ 'model_display_name',\n 'transformations',\n ],\n\ \ )(\n data_source_csv_filenames,\n data_source_bigquery_table_path,\n\ \ model_display_name,\n transformations,\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -7158,7 +6925,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-string-not-empty: container: args: @@ -7224,7 +6991,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-table-to-uri-2: container: args: @@ -7260,7 +7027,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -7305,7 +7072,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: The AutoML Forecasting pipeline. name: learn-to-learn-forecasting diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/sequence_to_sequence_forecasting_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/sequence_to_sequence_forecasting_pipeline.yaml index 67203a24b4..7ade233025 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/sequence_to_sequence_forecasting_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/sequence_to_sequence_forecasting_pipeline.yaml @@ -76,16 +76,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -100,9 +97,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -113,11 +108,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -157,10 +150,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-ensemble-2: executorLabel: exec-automl-forecasting-ensemble-2 @@ -170,16 +161,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -194,9 +182,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -207,11 +193,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -251,10 +235,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-1-tuner: executorLabel: exec-automl-forecasting-stage-1-tuner @@ -269,9 +251,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -284,9 +264,7 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the hyperparameter tuning should - - run.' + description: Number of hours the hyperparameter tuning should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -300,18 +278,16 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run hyperparameter tuning. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -322,22 +298,14 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "activation","categorical_value_spec": {"values": - - ["tanh"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "activation","categorical_value_spec": + {"values": ["tanh"]}}]' isOptional: true parameterType: LIST worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -349,11 +317,8 @@ components: description: The trained model and architectures. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-2-tuner: executorLabel: exec-automl-forecasting-stage-2-tuner @@ -368,16 +333,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The forecasting example gen - - metadata.' + description: The forecasting example gen metadata. transform_output: artifactType: schemaTitle: system.Artifact @@ -387,14 +348,11 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Path to the json of hyperparameter - - tuning results to use when evaluating models.' + description: Path to the json of hyperparameter tuning results to use when + evaluating models. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -408,9 +366,8 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model.' + description: Number of selected trials. The number of weak learners in the + final model. parameterType: NUMBER_INTEGER project: description: Project to run stage 2 tuner. @@ -423,13 +380,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -441,11 +393,8 @@ components: description: The trained (private) model artifact paths and their hyperparameters. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-finalizer: executorLabel: exec-automl-tabular-finalizer @@ -468,10 +417,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-calculate-training-parameters: executorLabel: exec-calculate-training-parameters @@ -965,6 +912,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -1678,6 +1628,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -2727,159 +2680,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -2889,24 +2808,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -2922,9 +2839,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -2939,17 +2854,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -2999,18 +2908,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -3020,9 +2924,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -3055,67 +2957,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -3125,9 +3006,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -3138,25 +3018,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -3180,264 +3055,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -3471,11 +3294,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -3496,36 +3317,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-finalize-eval-quantile-parameters: executorLabel: exec-finalize-eval-quantile-parameters @@ -5543,16 +5356,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -5560,9 +5369,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -5572,19 +5379,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -5599,76 +5399,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -5678,33 +5463,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -5714,45 +5490,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -5788,7 +5555,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5822,7 +5589,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5857,11 +5624,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_1_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--reduce_search_space_mode=", "{{$.inputs.parameters[''reduce_search_space_mode'']}}", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", @@ -5900,11 +5667,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_2_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", "\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -5943,7 +5710,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -6267,8 +6034,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -6285,7 +6052,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -6455,10 +6222,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6491,10 +6258,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6527,7 +6294,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-predictions-column-2: container: args: @@ -6556,7 +6323,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-importer: importer: artifactUri: @@ -7002,7 +6769,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-2: container: args: @@ -7031,7 +6798,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-set-optional-inputs: container: args: @@ -7094,7 +6861,7 @@ deploymentSpec: \ 'model_display_name',\n 'transformations',\n ],\n\ \ )(\n data_source_csv_filenames,\n data_source_bigquery_table_path,\n\ \ model_display_name,\n transformations,\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -7140,7 +6907,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-string-not-empty: container: args: @@ -7206,7 +6973,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-table-to-uri-2: container: args: @@ -7242,7 +7009,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -7287,7 +7054,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: The Sequence to Sequence (Seq2Seq) Forecasting pipeline. name: sequence-to-sequence-forecasting diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/temporal_fusion_transformer_forecasting_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/temporal_fusion_transformer_forecasting_pipeline.yaml index bf60e00ae7..9473c40662 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/temporal_fusion_transformer_forecasting_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/temporal_fusion_transformer_forecasting_pipeline.yaml @@ -75,16 +75,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -99,9 +96,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -112,11 +107,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -156,10 +149,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-ensemble-2: executorLabel: exec-automl-forecasting-ensemble-2 @@ -169,16 +160,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -193,9 +181,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -206,11 +192,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -250,10 +234,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-1-tuner: executorLabel: exec-automl-forecasting-stage-1-tuner @@ -268,9 +250,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -283,9 +263,7 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the hyperparameter tuning should - - run.' + description: Number of hours the hyperparameter tuning should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -299,18 +277,16 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run hyperparameter tuning. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -321,22 +297,14 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "activation","categorical_value_spec": {"values": - - ["tanh"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "activation","categorical_value_spec": + {"values": ["tanh"]}}]' isOptional: true parameterType: LIST worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -348,11 +316,8 @@ components: description: The trained model and architectures. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-2-tuner: executorLabel: exec-automl-forecasting-stage-2-tuner @@ -367,16 +332,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The forecasting example gen - - metadata.' + description: The forecasting example gen metadata. transform_output: artifactType: schemaTitle: system.Artifact @@ -386,14 +347,11 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Path to the json of hyperparameter - - tuning results to use when evaluating models.' + description: Path to the json of hyperparameter tuning results to use when + evaluating models. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -407,9 +365,8 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model.' + description: Number of selected trials. The number of weak learners in the + final model. parameterType: NUMBER_INTEGER project: description: Project to run stage 2 tuner. @@ -422,13 +379,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -440,11 +392,8 @@ components: description: The trained (private) model artifact paths and their hyperparameters. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-finalizer: executorLabel: exec-automl-tabular-finalizer @@ -467,10 +416,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-calculate-training-parameters: executorLabel: exec-calculate-training-parameters @@ -964,6 +911,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -1677,6 +1627,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -2720,159 +2673,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -2882,24 +2801,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -2915,9 +2832,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -2932,17 +2847,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -2992,18 +2901,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -3013,9 +2917,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -3048,67 +2950,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -3118,9 +2999,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -3131,25 +3011,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -3173,264 +3048,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -3464,11 +3287,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -3489,36 +3310,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-finalize-eval-quantile-parameters: executorLabel: exec-finalize-eval-quantile-parameters @@ -5536,16 +5349,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -5553,9 +5362,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -5565,19 +5372,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -5592,76 +5392,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -5671,33 +5456,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -5707,45 +5483,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -5781,7 +5548,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5815,7 +5582,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5850,11 +5617,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_1_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--reduce_search_space_mode=", "{{$.inputs.parameters[''reduce_search_space_mode'']}}", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", @@ -5893,11 +5660,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_2_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", "\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -5936,7 +5703,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -6260,8 +6027,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -6278,7 +6045,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -6448,10 +6215,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6484,10 +6251,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6520,7 +6287,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-predictions-column-2: container: args: @@ -6549,7 +6316,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-importer: importer: artifactUri: @@ -6995,7 +6762,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-2: container: args: @@ -7024,7 +6791,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-set-optional-inputs: container: args: @@ -7087,7 +6854,7 @@ deploymentSpec: \ 'model_display_name',\n 'transformations',\n ],\n\ \ )(\n data_source_csv_filenames,\n data_source_bigquery_table_path,\n\ \ model_display_name,\n transformations,\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -7133,7 +6900,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-string-not-empty: container: args: @@ -7199,7 +6966,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-table-to-uri-2: container: args: @@ -7235,7 +7002,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -7280,7 +7047,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: The Temporal Fusion Transformer (TFT) Forecasting pipeline. name: temporal-fusion-transformer-forecasting diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/time_series_dense_encoder_forecasting_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/time_series_dense_encoder_forecasting_pipeline.yaml index e8d94aa004..94e7ee5f34 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/time_series_dense_encoder_forecasting_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/forecasting/time_series_dense_encoder_forecasting_pipeline.yaml @@ -78,16 +78,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -102,9 +99,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -115,11 +110,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -159,10 +152,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-ensemble-2: executorLabel: exec-automl-forecasting-ensemble-2 @@ -172,16 +163,13 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. instance_schema_path: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The path to the instance schema, - - describing the input data for the tf_model at serving time.' + description: The path to the instance schema, describing the input data + for the tf_model at serving time. metadata: artifactType: schemaTitle: system.Artifact @@ -196,9 +184,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. parameters: encryption_spec_key_name: defaultValue: '' @@ -209,11 +195,9 @@ components: description: Region to run the job in. parameterType: STRING prediction_image_uri: - description: 'URI of the Docker image to be used as the - - container for serving predictions. This URI must identify an image in - - Artifact Registry or Container Registry.' + description: URI of the Docker image to be used as the container for serving + predictions. This URI must identify an image in Artifact Registry or Container + Registry. parameterType: STRING project: description: Project to run the job in. @@ -253,10 +237,8 @@ components: explanations. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-1-tuner: executorLabel: exec-automl-forecasting-stage-1-tuner @@ -271,9 +253,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -286,9 +266,7 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the hyperparameter tuning should - - run.' + description: Number of hours the hyperparameter tuning should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -302,18 +280,16 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run hyperparameter tuning. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -324,22 +300,14 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "activation","categorical_value_spec": {"values": - - ["tanh"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "activation","categorical_value_spec": + {"values": ["tanh"]}}]' isOptional: true parameterType: LIST worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -351,11 +319,8 @@ components: description: The trained model and architectures. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-forecasting-stage-2-tuner: executorLabel: exec-automl-forecasting-stage-2-tuner @@ -370,16 +335,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The forecasting example gen - - metadata.' + description: The forecasting example gen metadata. transform_output: artifactType: schemaTitle: system.Artifact @@ -389,14 +350,11 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Path to the json of hyperparameter - - tuning results to use when evaluating models.' + description: Path to the json of hyperparameter tuning results to use when + evaluating models. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE encryption_spec_key_name: defaultValue: '' @@ -410,9 +368,8 @@ components: description: Number of parallel training trials. parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model.' + description: Number of selected trials. The number of weak learners in the + final model. parameterType: NUMBER_INTEGER project: description: Project to run stage 2 tuner. @@ -425,13 +382,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -443,11 +395,8 @@ components: description: The trained (private) model artifact paths and their hyperparameters. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-finalizer: executorLabel: exec-automl-tabular-finalizer @@ -470,10 +419,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-calculate-training-parameters: executorLabel: exec-calculate-training-parameters @@ -971,6 +918,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -1689,6 +1639,9 @@ components: componentInputParameter: pipelinechannel--evaluation_dataflow_starting_num_workers encryption_spec_key_name: componentInputParameter: pipelinechannel--encryption_spec_key_name + force_runner_mode: + runtimeValue: + constant: Dataflow location: componentInputParameter: pipelinechannel--location predictions_format: @@ -2745,159 +2698,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -2907,24 +2826,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -2940,9 +2857,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -2957,17 +2872,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -3017,18 +2926,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -3038,9 +2942,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -3073,67 +2975,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -3143,9 +3024,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -3156,25 +3036,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -3198,264 +3073,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -3489,11 +3312,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -3514,36 +3335,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-finalize-eval-quantile-parameters: executorLabel: exec-finalize-eval-quantile-parameters @@ -5561,16 +5374,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -5578,9 +5387,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -5590,19 +5397,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -5617,76 +5417,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -5696,33 +5481,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -5732,45 +5508,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -5806,7 +5573,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5840,7 +5607,7 @@ deploymentSpec: - '{"display_name": "automl-forecasting-ensemble-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}", "encryption_spec": {"kms_key_name": "{{$.inputs.parameters[''encryption_spec_key_name'']}}"}, "job_spec": {"worker_pool_specs": [{"replica_count": 1, "machine_spec": - {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + {"machine_type": "n1-highmem-8"}, "container_spec": {"image_uri": "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "args": ["forecasting_mp_ensemble", "--transform_output_path={{$.inputs.artifacts[''transform_output''].uri}}", "--error_file_path={{$.inputs.parameters[''root_dir'']}}/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb", "--metadata_path={{$.inputs.artifacts[''metadata''].uri}}", "--tuning_result_input_path={{$.inputs.artifacts[''tuning_result_input''].uri}}", @@ -5875,11 +5642,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_1_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--reduce_search_space_mode=", "{{$.inputs.parameters[''reduce_search_space_mode'']}}", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", @@ -5918,11 +5685,11 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"args\": [\"forecasting_mp_l2l_stage_2_tuner", "\", \"--region=", "{{$.inputs.parameters[''location'']}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--training_docker_uri=", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20231029_0125", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/forecasting-training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}", "\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train", "\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -5961,7 +5728,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -6285,8 +6052,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -6303,7 +6070,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -6473,10 +6240,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6509,10 +6276,10 @@ deploymentSpec: Returns the prediction image corresponding to the given model type.\"\"\"\ \n # Keys come from AutoMlTimeSeriesForecastingTrainSpec.\n # The URIs\ \ must be hardcoded without any breaks in the code so string\n # replacement\ - \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20231029_0125',\n\ - \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20231029_0125',\n\ - \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20231029_0125',\n\ - \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20231029_0125',\n\ + \ will work correctly.\n images = {\n 'l2l': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-l2l:20240119_0125',\n\ + \ 'seq2seq': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-seq2seq:20240119_0125',\n\ + \ 'tft': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tft:20240119_0125',\n\ + \ 'tide': 'us-docker.pkg.dev/vertex-ai/automl-tabular/forecasting-prediction-server-tide:20240119_0125',\n\ \ }\n if model_type not in images:\n raise ValueError(\n f'Invalid\ \ forecasting model type: {model_type}. Valid options are: '\n f'{images.keys()}.'\n\ \ )\n return images[model_type]\n\n" @@ -6545,7 +6312,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-predictions-column-2: container: args: @@ -6574,7 +6341,7 @@ deploymentSpec: \ str) -> str:\n \"\"\"Generates the BP output's target column name.\"\"\ \"\n if forecasting_type == 'quantile':\n return f'predicted_{target_column}.quantile_predictions'\n\ \ return f'predicted_{target_column}.value'\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-importer: importer: artifactUri: @@ -7020,7 +6787,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-2: container: args: @@ -7049,7 +6816,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-set-optional-inputs: container: args: @@ -7112,7 +6879,7 @@ deploymentSpec: \ 'model_display_name',\n 'transformations',\n ],\n\ \ )(\n data_source_csv_filenames,\n data_source_bigquery_table_path,\n\ \ model_display_name,\n transformations,\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -7158,7 +6925,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-string-not-empty: container: args: @@ -7224,7 +6991,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-table-to-uri-2: container: args: @@ -7260,7 +7027,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -7305,7 +7072,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: The Timeseries Dense Encoder (TiDE) Forecasting pipeline. name: time-series-dense-encoder-forecasting diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/auto_feature_engineering.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/auto_feature_engineering.py index a893d41552..c447bb1cb2 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/auto_feature_engineering.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/auto_feature_engineering.py @@ -65,7 +65,7 @@ def automated_feature_engineering( ' 1, "machine_spec": {"machine_type": "n1-standard-16"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "args": ["feature_engineering", "--project=', project, '", "--location=', location, '", "--data_source_bigquery_table_path=', data_source_bigquery_table_path, diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_feature_selection_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_feature_selection_pipeline.yaml index c429165b9c..80187c3af3 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_feature_selection_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_feature_selection_pipeline.yaml @@ -112,15 +112,13 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. @@ -133,13 +131,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -154,10 +147,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-cv-trainer-2: executorLabel: exec-automl-tabular-cv-trainer-2 @@ -200,15 +191,13 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. @@ -221,13 +210,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -242,10 +226,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble: executorLabel: exec-automl-tabular-ensemble @@ -260,9 +242,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -277,18 +257,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -298,11 +274,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -348,10 +321,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble-2: executorLabel: exec-automl-tabular-ensemble-2 @@ -366,9 +337,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -383,18 +352,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -404,11 +369,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -454,10 +416,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble-3: executorLabel: exec-automl-tabular-ensemble-3 @@ -472,9 +432,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -489,18 +447,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -510,11 +464,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -560,10 +511,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-finalizer: executorLabel: exec-automl-tabular-finalizer @@ -586,10 +535,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-infra-validator: executorLabel: exec-automl-tabular-infra-validator @@ -599,9 +546,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-infra-validator-2: executorLabel: exec-automl-tabular-infra-validator-2 inputDefinitions: @@ -610,9 +555,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-infra-validator-3: executorLabel: exec-automl-tabular-infra-validator-3 inputDefinitions: @@ -621,9 +564,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-stage-1-tuner: executorLabel: exec-automl-tabular-stage-1-tuner inputDefinitions: @@ -642,9 +583,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -657,15 +596,11 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE disable_early_stopping: defaultValue: false - description: 'True if disable early stopping. Default - - value is false.' + description: True if disable early stopping. Default value is false. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -681,24 +616,21 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -706,9 +638,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN single_run_max_secs: @@ -716,11 +646,8 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "model_type","categorical_value_spec": {"values": - - ["nn"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "model_type","categorical_value_spec": + {"values": ["nn"]}}]' isOptional: true parameterType: LIST tune_feature_selection_rate: @@ -729,13 +656,8 @@ components: parameterType: BOOLEAN worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -750,10 +672,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-stage-1-tuner-2: executorLabel: exec-automl-tabular-stage-1-tuner-2 @@ -773,9 +693,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -788,15 +706,11 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE disable_early_stopping: defaultValue: false - description: 'True if disable early stopping. Default - - value is false.' + description: True if disable early stopping. Default value is false. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -812,24 +726,21 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -837,9 +748,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN single_run_max_secs: @@ -847,11 +756,8 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "model_type","categorical_value_spec": {"values": - - ["nn"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "model_type","categorical_value_spec": + {"values": ["nn"]}}]' isOptional: true parameterType: LIST tune_feature_selection_rate: @@ -860,13 +766,8 @@ components: parameterType: BOOLEAN worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -881,10 +782,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-transform: executorLabel: exec-automl-tabular-transform @@ -918,46 +817,36 @@ components: parameters: dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More - - details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -1002,10 +891,8 @@ components: description: The transform output artifact. parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-transform-2: executorLabel: exec-automl-tabular-transform-2 @@ -1039,46 +926,36 @@ components: parameters: dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More - - details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -1123,10 +1000,8 @@ components: description: The transform output artifact. parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-bool-identity: executorLabel: exec-bool-identity @@ -8411,73 +8286,54 @@ components: parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More - - details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: defaultValue: '' - description: 'Customer-managed encryption key. - - If this is set, then all resources will be encrypted with the provided - - encryption key. data_source(Dataset): The input dataset artifact which - - references csv, BigQuery, or TF Records. target_column_name(str): Target - - column name of the input dataset.' + description: 'Customer-managed encryption key. If this is set, then all + resources will be encrypted with the provided encryption key. data_source(Dataset): + The input dataset artifact which references csv, BigQuery, or TF Records. + target_column_name(str): Target column name of the input dataset.' isOptional: true parameterType: STRING location: - description: 'Location for running the feature selection. If not set, - - default to us-central1.' + description: Location for running the feature selection. If not set, default + to us-central1. parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'number of features to select by the - - algorithm. If not set, default to 1000.' + description: number of features to select by the algorithm. If not set, + default to 1000. isOptional: true parameterType: NUMBER_INTEGER prediction_type: @@ -8506,11 +8362,8 @@ components: description: A json array of selected feature names. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-tabular-stats-and-example-gen: executorLabel: exec-tabular-stats-and-example-gen @@ -8534,48 +8387,36 @@ components: parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More - - details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN enable_probabilistic_inference: @@ -8588,44 +8429,36 @@ components: isOptional: true parameterType: STRING location: - description: 'Location for running dataset statistics and example - - generation.' + description: Location for running dataset statistics and example generation. parameterType: STRING optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE predefined_split_key: @@ -8633,14 +8466,10 @@ components: isOptional: true parameterType: STRING prediction_type: - description: 'The prediction type. Supported values: - - "classification", "regression".' + description: 'The prediction type. Supported values: "classification", "regression".' parameterType: STRING project: - description: 'Project to run dataset statistics and example - - generation.' + description: Project to run dataset statistics and example generation. parameterType: STRING quantiles: defaultValue: [] @@ -8655,9 +8484,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN stratified_split_key: @@ -8680,21 +8507,14 @@ components: isOptional: true parameterType: NUMBER_DOUBLE transformations: - description: 'Quote escaped JSON string for transformations. Each - - transformation will apply transform function to given input column. And - - the result will be used for training. When creating transformation for - - BigQuery Struct column, the column should be flattened using "." as the - - delimiter.' + description: Quote escaped JSON string for transformations. Each transformation + will apply transform function to given input column. And the result will + be used for training. When creating transformation for BigQuery Struct + column, the column should be flattened using "." as the delimiter. parameterType: STRING transformations_path: defaultValue: '' - description: 'Path to a GCS file containing JSON - - string for transformations.' + description: Path to a GCS file containing JSON string for transformations. isOptional: true parameterType: STRING validation_fraction: @@ -8748,10 +8568,8 @@ components: description: The downsampled test split JSON object. parameterType: LIST gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING test_split_json: description: The test split JSON object. @@ -8804,9 +8622,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_cv_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -8847,9 +8665,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_cv_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -8890,7 +8708,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -8902,7 +8720,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -8931,7 +8749,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -8943,7 +8761,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -8972,7 +8790,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -8984,7 +8802,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -9013,7 +8831,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -9028,7 +8846,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -9037,7 +8855,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -9046,7 +8864,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -9066,9 +8884,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_stage_1_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--feature_selection_result_path=", "{{$.inputs.artifacts[''feature_ranking''].uri}}", "\", \"--disable_early_stopping=", "{{$.inputs.parameters[''disable_early_stopping'']}}", "\", \"--tune_feature_selection_rate=", "{{$.inputs.parameters[''tune_feature_selection_rate'']}}", @@ -9113,9 +8931,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_stage_1_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--feature_selection_result_path=", "{{$.inputs.artifacts[''feature_ranking''].uri}}", "\", \"--disable_early_stopping=", "{{$.inputs.parameters[''disable_early_stopping'']}}", "\", \"--tune_feature_selection_rate=", "{{$.inputs.parameters[''tune_feature_selection_rate'']}}", @@ -9160,7 +8978,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"transform\", \"--is_mp=true\", \"--transform_output_artifact_path=", "{{$.outputs.artifacts[''transform_output''].uri}}", "\", \"--transform_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/transform\", @@ -9181,7 +8999,7 @@ deploymentSpec: \"--dataflow_tmp_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp\", \"--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}", "\", \"--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}", - "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", + "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "\", \"--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}", "\", \"--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}", "\", \"--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}", @@ -9212,7 +9030,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"transform\", \"--is_mp=true\", \"--transform_output_artifact_path=", "{{$.outputs.artifacts[''transform_output''].uri}}", "\", \"--transform_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/transform\", @@ -9233,7 +9051,7 @@ deploymentSpec: \"--dataflow_tmp_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp\", \"--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}", "\", \"--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}", - "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", + "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "\", \"--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}", "\", \"--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}", "\", \"--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}", @@ -10499,7 +10317,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-2: container: args: @@ -10528,7 +10346,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-3: container: args: @@ -10557,7 +10375,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-purge-unused-features: container: args: @@ -10728,7 +10546,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"feature_selection\", \"--data_source=", "{{$.inputs.artifacts[''data_source''].uri}}", "\", \"--target_column=", "{{$.inputs.parameters[''target_column_name'']}}", "\", \"--prediction_type=", "{{$.inputs.parameters[''prediction_type'']}}", @@ -10741,7 +10559,7 @@ deploymentSpec: \"--dataflow_staging_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_staging\", \"--dataflow_tmp_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp\", \"--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}", - "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", + "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "\", \"--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}", "\", \"--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}", "\", \"--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}", @@ -10774,7 +10592,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"stats_generator\",", "\"--train_spec={\\\"prediction_type\\\": \\\"", "{{$.inputs.parameters[''prediction_type'']}}", "\\\", \\\"target_column\\\": \\\"", "{{$.inputs.parameters[''target_column_name'']}}", "\\\", \\\"optimization_objective\\\": @@ -10807,7 +10625,7 @@ deploymentSpec: \"--dataflow_staging_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_staging\", \"--dataflow_tmp_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp\", \"--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}", - "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", + "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "\", \"--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}", "\", \"--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}", "\", \"--dataflow_kms_key=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_v2_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_v2_pipeline.yaml index 861946ed5c..5ffac83a46 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_v2_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/automl_tabular_v2_pipeline.yaml @@ -120,15 +120,13 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. @@ -141,13 +139,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -162,10 +155,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-cv-trainer-2: executorLabel: exec-automl-tabular-cv-trainer-2 @@ -208,15 +199,13 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. @@ -229,13 +218,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -250,10 +234,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble: executorLabel: exec-automl-tabular-ensemble @@ -268,9 +250,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -285,18 +265,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -306,11 +282,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -356,10 +329,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble-2: executorLabel: exec-automl-tabular-ensemble-2 @@ -374,9 +345,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -391,18 +360,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -412,11 +377,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -462,10 +424,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble-3: executorLabel: exec-automl-tabular-ensemble-3 @@ -480,9 +440,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -497,18 +455,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -518,11 +472,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -568,10 +519,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-finalizer: executorLabel: exec-automl-tabular-finalizer @@ -594,10 +543,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-infra-validator: executorLabel: exec-automl-tabular-infra-validator @@ -607,9 +554,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-infra-validator-2: executorLabel: exec-automl-tabular-infra-validator-2 inputDefinitions: @@ -618,9 +563,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-infra-validator-3: executorLabel: exec-automl-tabular-infra-validator-3 inputDefinitions: @@ -629,9 +572,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-stage-1-tuner: executorLabel: exec-automl-tabular-stage-1-tuner inputDefinitions: @@ -650,9 +591,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -665,15 +604,11 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE disable_early_stopping: defaultValue: false - description: 'True if disable early stopping. Default - - value is false.' + description: True if disable early stopping. Default value is false. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -689,24 +624,21 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -714,9 +646,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN single_run_max_secs: @@ -724,11 +654,8 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "model_type","categorical_value_spec": {"values": - - ["nn"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "model_type","categorical_value_spec": + {"values": ["nn"]}}]' isOptional: true parameterType: LIST tune_feature_selection_rate: @@ -737,13 +664,8 @@ components: parameterType: BOOLEAN worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -758,10 +680,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-stage-1-tuner-2: executorLabel: exec-automl-tabular-stage-1-tuner-2 @@ -781,9 +701,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -796,15 +714,11 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE disable_early_stopping: defaultValue: false - description: 'True if disable early stopping. Default - - value is false.' + description: True if disable early stopping. Default value is false. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -820,24 +734,21 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -845,9 +756,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN single_run_max_secs: @@ -855,11 +764,8 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "model_type","categorical_value_spec": {"values": - - ["nn"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "model_type","categorical_value_spec": + {"values": ["nn"]}}]' isOptional: true parameterType: LIST tune_feature_selection_rate: @@ -868,13 +774,8 @@ components: parameterType: BOOLEAN worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -889,10 +790,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-bool-identity: executorLabel: exec-bool-identity @@ -3788,79 +3687,55 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - ''projectId.datasetId'' format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - ''vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}''. - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in 'projectId.datasetId' format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called 'vertex_feature_transform_engine_staging_{location.replace('-', + '_')}'. All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING bigquery_train_full_table_uri: - description: 'BigQuery full table id for our - - train split output by pre-distillation FTE with soft target included.' + description: BigQuery full table id for our train split output by pre-distillation + FTE with soft target included. parameterType: STRING bigquery_validate_full_table_uri: - description: 'BigQuery full table id for our - - validation split output by pre-distillation FTE with soft target - - included.' + description: BigQuery full table id for our validation split output by pre-distillation + FTE with soft target included. parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -3872,9 +3747,8 @@ components: description: Location for the created GCP services. parameterType: STRING prediction_type: - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". parameterType: STRING project: description: Project to run feature transform engine. @@ -3886,9 +3760,8 @@ components: description: Target column of input data. parameterType: STRING transform_config_path: - description: 'Path to the transform config output by the - - pre-distillation FTE component.' + description: Path to the transform config output by the pre-distillation + FTE component. parameterType: STRING weight_column: defaultValue: '' @@ -3909,11 +3782,8 @@ components: description: The transform output artifact. parameters: gcp_resources: - description: 'GCP resources created by this component. - - For more details, see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-exit-handler-1: dag: @@ -4776,159 +4646,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -4938,24 +4774,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -4971,9 +4805,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -4988,17 +4820,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -5048,18 +4874,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -5069,9 +4890,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -5104,67 +4923,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -5174,9 +4972,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -5187,25 +4984,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -5229,264 +5021,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -5520,11 +5260,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -5545,36 +5283,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-get-bigquery-destination-output-uri: executorLabel: exec-get-bigquery-destination-output-uri @@ -9327,16 +9057,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -9344,9 +9070,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -9356,19 +9080,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -9383,76 +9100,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -9462,33 +9164,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -9498,45 +9191,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -9563,16 +9247,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -9580,9 +9260,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -9592,19 +9270,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -9619,76 +9290,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -9698,33 +9354,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -9734,45 +9381,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -9809,9 +9447,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_cv_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -9852,9 +9490,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_cv_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -9895,7 +9533,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -9907,7 +9545,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -9936,7 +9574,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -9948,7 +9586,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -9977,7 +9615,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -9989,7 +9627,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -10018,7 +9656,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -10033,7 +9671,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -10042,7 +9680,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -10051,7 +9689,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -10071,9 +9709,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_stage_1_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--feature_selection_result_path=", "{{$.inputs.artifacts[''feature_ranking''].uri}}", "\", \"--disable_early_stopping=", "{{$.inputs.parameters[''disable_early_stopping'']}}", "\", \"--tune_feature_selection_rate=", "{{$.inputs.parameters[''tune_feature_selection_rate'']}}", @@ -10118,9 +9756,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_stage_1_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--feature_selection_result_path=", "{{$.inputs.artifacts[''feature_ranking''].uri}}", "\", \"--disable_early_stopping=", "{{$.inputs.parameters[''disable_early_stopping'']}}", "\", \"--tune_feature_selection_rate=", "{{$.inputs.parameters[''tune_feature_selection_rate'']}}", @@ -10462,14 +10100,14 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' - '{"Concat": ["--dataflow_service_account=", "{{$.inputs.parameters[''dataflow_service_account'']}}"]}' - '{"Concat": ["--dataflow_kms_key=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - '{"Concat": ["--gcp_resources_path=", "{{$.outputs.parameters[''gcp_resources''].output_file}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -10716,8 +10354,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -10734,7 +10372,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -10920,7 +10558,7 @@ deploymentSpec: \ )\n\n return collections.namedtuple(\n 'Outputs',\n [\n \ \ 'transform_config_path',\n ],\n )(\n transform_config_path,\n\ \ )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-importer: importer: artifactUri: @@ -11699,7 +11337,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-2: container: args: @@ -11728,7 +11366,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-3: container: args: @@ -11757,7 +11395,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-set-optional-inputs: container: args: @@ -11814,7 +11452,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -11860,7 +11498,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-split-materialized-data-2: container: args: @@ -11906,7 +11544,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-string-not-empty: container: args: @@ -11981,7 +11619,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 exec-training-configurator-and-validator-2: container: args: @@ -12026,7 +11664,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: The AutoML Tabular pipeline v2. name: automl-tabular-v2 diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/distillation_stage_feature_transform_engine.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/distillation_stage_feature_transform_engine.py index 2948091c54..e611cf5a07 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/distillation_stage_feature_transform_engine.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/distillation_stage_feature_transform_engine.py @@ -51,31 +51,33 @@ def distillation_stage_feature_transform_engine( transform the input datasets with predicted outputs included (soft targets). Args: - root_dir: The Cloud Storage location to store the output. - project: Project to run feature transform engine. - location: Location for the created GCP services. - transform_config_path: Path to the transform config output by the pre-distillation FTE component. - bigquery_train_full_table_uri: BigQuery full table id for our train split output by pre-distillation FTE with soft target included. - bigquery_validate_full_table_uri: BigQuery full table id for our validation split output by pre-distillation FTE with soft target included. - target_column: Target column of input data. prediction_type (str): Model prediction type. One of "classification", "regression", "time_series". - bigquery_staging_full_dataset_id: Dataset in 'projectId.datasetId' format for storing intermediate-FTE BigQuery tables. If the specified dataset does not exist in BigQuery, FTE will create the dataset. If no bigquery_staging_full_dataset_id is specified, all intermediate tables will be stored in a dataset created under the provided project in the input data source's location during FTE execution called 'vertex_feature_transform_engine_staging_{location.replace('-', '_')}'. All tables generated by FTE will have a 30 day TTL. - weight_column: Weight column of input data. - dataflow_machine_type: The machine type used for dataflow jobs. If not set, default to n1-standard-16. - dataflow_max_num_workers: The number of workers to run the dataflow job. If not set, default to 25. - dataflow_disk_size_gb: The disk size, in gigabytes, to use on each Dataflow worker instance. If not set, default to 40. - dataflow_subnetwork: Dataflow's fully qualified subnetwork name, when empty the default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications dataflow_use_public_ips (Optional[bool]): Specifies whether Dataflow workers use public IP addresses. - dataflow_service_account: Custom service account to run Dataflow jobs. - encryption_spec_key_name: Customer-managed encryption key. + root_dir: The Cloud Storage location to store the output. + project: Project to run feature transform engine. + location: Location for the created GCP services. + transform_config_path: Path to the transform config output by the pre-distillation FTE component. + bigquery_train_full_table_uri: BigQuery full table id for our train split output by pre-distillation FTE with soft target included. + bigquery_validate_full_table_uri: BigQuery full table id for our validation split output by pre-distillation FTE with soft target included. + target_column: Target column of input data. + prediction_type: Model prediction type. One of "classification", "regression", "time_series". + bigquery_staging_full_dataset_id: Dataset in 'projectId.datasetId' format for storing intermediate-FTE BigQuery tables. If the specified dataset does not exist in BigQuery, FTE will create the dataset. If no bigquery_staging_full_dataset_id is specified, all intermediate tables will be stored in a dataset created under the provided project in the input data source's location during FTE execution called 'vertex_feature_transform_engine_staging_{location.replace('-', '_')}'. All tables generated by FTE will have a 30 day TTL. + weight_column: Weight column of input data. + dataflow_machine_type: The machine type used for dataflow jobs. If not set, default to n1-standard-16. + dataflow_max_num_workers: The number of workers to run the dataflow job. If not set, default to 25. + dataflow_disk_size_gb: The disk size, in gigabytes, to use on each Dataflow worker instance. If not set, default to 40. + dataflow_subnetwork: Dataflow's fully qualified subnetwork name, when empty the default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications + dataflow_use_public_ips: Specifies whether Dataflow workers use public IP addresses. + dataflow_service_account: Custom service account to run Dataflow jobs. + encryption_spec_key_name: Customer-managed encryption key. Returns: - materialized_data: The materialized dataset. - transform_output: The transform output artifact. - gcp_resources: GCP resources created by this component. For more details, see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. + materialized_data: The materialized dataset. + transform_output: The transform output artifact. + gcp_resources: GCP resources created by this component. For more details, see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. """ # fmt: on return dsl.ContainerSpec( - image='us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125', + image='us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125', command=[], args=[ 'distillation_stage_feature_transform_engine', @@ -183,7 +185,7 @@ def distillation_stage_feature_transform_engine( dataflow_machine_type, ] ), - '--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125', + '--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125', dsl.ConcatPlaceholder( items=[ '--dataflow_disk_size_gb=', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection.py index e94dd0d312..c17cddf29f 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection.py @@ -100,7 +100,7 @@ def tabular_feature_ranking_and_selection( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "args": ["feature_selection", "--data_source=', data_source.uri, '", "--target_column=', @@ -137,7 +137,7 @@ def tabular_feature_ranking_and_selection( ), dataflow_max_num_workers, '", "--dataflow_worker_container_image=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125', '", "--dataflow_machine_type=', dataflow_machine_type, '", "--dataflow_disk_size_gb=', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection_pipeline.yaml index 47697ad0a6..6082eebc9a 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_selection_pipeline.yaml @@ -47,159 +47,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -209,24 +175,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -242,9 +206,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -259,17 +221,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -319,18 +275,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -340,9 +291,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -375,67 +324,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -445,9 +373,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -458,25 +385,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -500,264 +422,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -791,11 +661,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -816,36 +684,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-training-configurator-and-validator: executorLabel: exec-training-configurator-and-validator @@ -855,16 +715,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -872,9 +728,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -884,19 +738,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -911,76 +758,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -990,33 +822,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -1026,45 +849,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -1169,8 +983,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -1187,7 +1001,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -1235,7 +1049,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: Defines pipeline for feature transform engine component. name: feature-selection diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_transform_engine.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_transform_engine.py index e1f3743061..82dc8f1115 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_transform_engine.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/feature_transform_engine.py @@ -308,7 +308,7 @@ def feature_transform_engine( # fmt: on return dsl.ContainerSpec( - image='us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125', + image='us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125', command=[], args=[ 'feature_transform_engine', @@ -637,8 +637,8 @@ def feature_transform_engine( dsl.ConcatPlaceholder( items=['--dataflow_machine_type=', dataflow_machine_type] ), - '--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125', - '--feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125', + '--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125', + '--feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125', dsl.ConcatPlaceholder( items=['--dataflow_disk_size_gb=', dataflow_disk_size_gb] ), diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job.py index bb317cd9c8..591b2b510d 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job.py @@ -158,7 +158,7 @@ def tabnet_hyperparameter_tuning_job( ', "disk_spec": ', training_disk_spec, ', "container_spec": {"image_uri":"', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20240119_0125', '", "args": ["--target_column=', target_column, '", "--weight_column=', @@ -166,7 +166,7 @@ def tabnet_hyperparameter_tuning_job( '", "--model_type=', prediction_type, '", "--prediction_docker_uri=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125', '", "--prediction_docker_uri_artifact_path=', prediction_docker_uri_output, '", "--baseline_path=', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job_pipeline.yaml index e52cad6501..7d5010a22d 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_hyperparameter_tuning_job_pipeline.yaml @@ -83,10 +83,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-infra-validator: executorLabel: exec-automl-tabular-infra-validator @@ -96,9 +94,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-bool-identity: executorLabel: exec-bool-identity inputDefinitions: @@ -836,159 +832,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -998,24 +960,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -1031,9 +991,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -1048,17 +1006,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -1108,18 +1060,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -1129,9 +1076,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -1164,67 +1109,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -1234,9 +1158,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -1247,25 +1170,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -1289,264 +1207,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -1580,11 +1446,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -1605,36 +1469,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-get-best-hyperparameter-tuning-job-trial: executorLabel: exec-get-best-hyperparameter-tuning-job-trial @@ -2630,16 +2486,13 @@ components: parameters: cache_data: defaultValue: auto - description: 'Whether to cache data or not. If set to - - ''auto'', caching is determined based on the dataset size.' + description: Whether to cache data or not. If set to 'auto', caching is + determined based on the dataset size. isOptional: true parameterType: STRING enable_profiler: defaultValue: false - description: 'Enables profiling and saves a trace - - during evaluation.' + description: Enables profiling and saves a trace during evaluation. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -2649,20 +2502,14 @@ components: parameterType: STRING eval_frequency_secs: defaultValue: 600.0 - description: 'Frequency at which evaluation and - - checkpointing will take place.' + description: Frequency at which evaluation and checkpointing will take place. isOptional: true parameterType: NUMBER_INTEGER eval_steps: defaultValue: 0.0 - description: 'Number of steps to run evaluation for. If not - - specified or negative, it means run evaluation on the whole validation - - dataset. If set to 0, it means run evaluation for a fixed number of - - samples.' + description: Number of steps to run evaluation for. If not specified or + negative, it means run evaluation on the whole validation dataset. If + set to 0, it means run evaluation for a fixed number of samples. isOptional: true parameterType: NUMBER_INTEGER location: @@ -2670,26 +2517,20 @@ components: parameterType: STRING max_failed_trial_count: defaultValue: 0.0 - description: 'The number of failed trials that - - need to be seen before failing the HyperparameterTuningJob. If set to - 0, - - Vertex AI decides how many trials must fail before the whole job fails.' + description: The number of failed trials that need to be seen before failing + the HyperparameterTuningJob. If set to 0, Vertex AI decides how many trials + must fail before the whole job fails. isOptional: true parameterType: NUMBER_INTEGER max_trial_count: description: The desired total number of trials. parameterType: NUMBER_INTEGER parallel_trial_count: - description: 'The desired number of trials to run - - in parallel.' + description: The desired number of trials to run in parallel. parameterType: NUMBER_INTEGER prediction_type: - description: 'The type of prediction the model is to - - produce. "classification" or "regression".' + description: The type of prediction the model is to produce. "classification" + or "regression". parameterType: STRING project: description: The GCP project that runs the pipeline components. @@ -2704,45 +2545,30 @@ components: parameterType: NUMBER_INTEGER study_spec_algorithm: defaultValue: ALGORITHM_UNSPECIFIED - description: 'The search algorithm specified for - - the study. One of ''ALGORITHM_UNSPECIFIED'', ''GRID_SEARCH'', or - - ''RANDOM_SEARCH''.' + description: The search algorithm specified for the study. One of 'ALGORITHM_UNSPECIFIED', + 'GRID_SEARCH', or 'RANDOM_SEARCH'. isOptional: true parameterType: STRING study_spec_measurement_selection_type: defaultValue: BEST_MEASUREMENT - description: 'Which measurement - - to use if/when the service automatically selects the final measurement - - from previously reported intermediate measurements. One of - - "BEST_MEASUREMENT" or "LAST_MEASUREMENT".' + description: Which measurement to use if/when the service automatically + selects the final measurement from previously reported intermediate measurements. + One of "BEST_MEASUREMENT" or "LAST_MEASUREMENT". isOptional: true parameterType: STRING study_spec_metric_goal: - description: 'Optimization goal of the metric, - - possible values: "MAXIMIZE", "MINIMIZE".' + description: 'Optimization goal of the metric, possible values: "MAXIMIZE", + "MINIMIZE".' parameterType: STRING study_spec_metric_id: - description: 'Metric to optimize, possible - - values: [ ''loss'', ''average_loss'', ''rmse'', ''mae'', ''mql'', ''accuracy'', - ''auc'', ''precision'', ''recall''].' + description: 'Metric to optimize, possible values: [ ''loss'', ''average_loss'', + ''rmse'', ''mae'', ''mql'', ''accuracy'', ''auc'', ''precision'', ''recall''].' parameterType: STRING study_spec_parameters_override: - description: 'List of dictionaries - - representing parameters to optimize. The dictionary key is the - - parameter_id, which is passed to training job as a command line - - argument, and the dictionary value is the parameter specification of the - - metric.' + description: List of dictionaries representing parameters to optimize. The + dictionary key is the parameter_id, which is passed to training job as + a command line argument, and the dictionary value is the parameter specification + of the metric. parameterType: LIST target_column: description: The target column name. @@ -2757,11 +2583,8 @@ components: training_machine_spec: defaultValue: machine_type: c2-standard-16 - description: 'The training machine - - spec. See https://cloud.google.com/compute/docs/machine-types for - - options.' + description: The training machine spec. See https://cloud.google.com/compute/docs/machine-types + for options. isOptional: true parameterType: STRUCT weight_column: @@ -2798,16 +2621,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -2815,9 +2634,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -2827,19 +2644,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -2854,76 +2664,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -2933,33 +2728,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -2969,45 +2755,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -3044,7 +2821,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -3059,7 +2836,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -3175,8 +2952,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -3193,7 +2970,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -3267,7 +3044,7 @@ deploymentSpec: \ = {\n 'instanceSchemaUri': instance_schema_uri,\n 'predictionSchemaUri':\ \ prediction_schema_uri,\n }\n unmanaged_container_model.uri = os.path.join(\n\ \ trials_dir, 'trial_{}'.format(best_trial['id']), 'model'\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-tabnet-study-spec-parameters: container: args: @@ -3783,7 +3560,7 @@ deploymentSpec: \ = ', '.join(extra_overrides)\n warnings.warn(\n f'The overrides\ \ {extra_override_str} were not found in the params and '\n 'will\ \ be ignored.'\n )\n\n return study_spec_parameters\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-model-batch-predict: container: args: @@ -4087,7 +3864,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -4133,7 +3910,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-tabnet-hyperparameter-tuning-job: container: args: @@ -4161,11 +3938,11 @@ deploymentSpec: ", \"trial_job_spec\": {\"worker_pool_specs\": [{\"replica_count\":\"", "1", "\", \"machine_spec\": ", "{{$.inputs.parameters[''training_machine_spec'']}}", ", \"disk_spec\": ", "{{$.inputs.parameters[''training_disk_spec'']}}", - ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20231029_0125", + ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20240119_0125", "\", \"args\": [\"--target_column=", "{{$.inputs.parameters[''target_column'']}}", "\", \"--weight_column=", "{{$.inputs.parameters[''weight_column'']}}", "\", \"--model_type=", "{{$.inputs.parameters[''prediction_type'']}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--prediction_docker_uri_artifact_path=", "{{$.outputs.parameters[''prediction_docker_uri_output''].output_file}}", "\", \"--baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--metadata_path=", "{{$.inputs.artifacts[''metadata''].uri}}", "\", @@ -4234,7 +4011,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: The TabNet built-in algorithm HyperparameterTuningJob pipeline. name: automl-tabular-tabnet-hyperparameter-tuning-job diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer.py index dae5a9c23e..4c098555f6 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer.py @@ -165,7 +165,7 @@ def tabnet_trainer( ', "disk_spec": ', training_disk_spec, ', "container_spec": {"image_uri":"', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20240119_0125', '", "args": ["--target_column=', target_column, '", "--weight_column=', @@ -173,7 +173,7 @@ def tabnet_trainer( '", "--model_type=', prediction_type, '", "--prediction_docker_uri=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125', '", "--baseline_path=', instance_baseline.uri, '", "--metadata_path=', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer_pipeline.yaml index 5d37139d95..fd08a353b2 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/tabnet_trainer_pipeline.yaml @@ -102,10 +102,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-infra-validator: executorLabel: exec-automl-tabular-infra-validator @@ -115,9 +113,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-bool-identity: executorLabel: exec-bool-identity inputDefinitions: @@ -865,159 +861,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -1027,24 +989,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -1060,9 +1020,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -1077,17 +1035,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -1137,18 +1089,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -1158,9 +1105,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -1193,67 +1138,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -1263,9 +1187,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -1276,25 +1199,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -1318,264 +1236,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -1609,11 +1475,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -1634,36 +1498,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-model-batch-predict: executorLabel: exec-model-batch-predict @@ -2585,9 +2441,8 @@ components: parameters: alpha_focal_loss: defaultValue: 0.25 - description: 'Alpha value (balancing factor) in - - focal_loss function. Only used for classification.' + description: Alpha value (balancing factor) in focal_loss function. Only + used for classification. isOptional: true parameterType: NUMBER_DOUBLE batch_momentum: @@ -2602,32 +2457,26 @@ components: parameterType: NUMBER_INTEGER batch_size_ratio: defaultValue: 0.25 - description: 'The ratio of virtual batch size (size - - of the ghost batch normalization) to batch size.' + description: The ratio of virtual batch size (size of the ghost batch normalization) + to batch size. isOptional: true parameterType: NUMBER_DOUBLE cache_data: defaultValue: auto - description: 'Whether to cache data or not. If set to - - ''auto'', caching is determined based on the dataset size.' + description: Whether to cache data or not. If set to 'auto', caching is + determined based on the dataset size. isOptional: true parameterType: STRING class_weight: defaultValue: 1.0 - description: 'The class weight is used to computes a - - weighted cross entropy which is helpful in classify imbalanced dataset. - - Only used for classification.' + description: The class weight is used to computes a weighted cross entropy + which is helpful in classify imbalanced dataset. Only used for classification. isOptional: true parameterType: NUMBER_DOUBLE decay_every: defaultValue: 100.0 - description: 'Number of iterations for periodically - - applying learning rate decaying.' + description: Number of iterations for periodically applying learning rate + decaying. isOptional: true parameterType: NUMBER_DOUBLE decay_rate: @@ -2637,9 +2486,7 @@ components: parameterType: NUMBER_DOUBLE enable_profiler: defaultValue: false - description: 'Enables profiling and saves a trace - - during evaluation.' + description: Enables profiling and saves a trace during evaluation. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -2649,43 +2496,32 @@ components: parameterType: STRING eval_frequency_secs: defaultValue: 600.0 - description: 'Frequency at which evaluation and - - checkpointing will take place.' + description: Frequency at which evaluation and checkpointing will take place. isOptional: true parameterType: NUMBER_INTEGER eval_steps: defaultValue: 0.0 - description: 'Number of steps to run evaluation for. If not - - specified or negative, it means run evaluation on the whole validation - - dataset. If set to 0, it means run evaluation for a fixed number of - - samples.' + description: Number of steps to run evaluation for. If not specified or + negative, it means run evaluation on the whole validation dataset. If + set to 0, it means run evaluation for a fixed number of samples. isOptional: true parameterType: NUMBER_INTEGER feature_dim: defaultValue: 64.0 - description: 'Dimensionality of the hidden representation - - in feature transformation block.' + description: Dimensionality of the hidden representation in feature transformation + block. isOptional: true parameterType: NUMBER_INTEGER feature_dim_ratio: defaultValue: 0.5 - description: 'The ratio of output dimension - - (dimensionality of the outputs of each decision step) to feature - - dimension.' + description: The ratio of output dimension (dimensionality of the outputs + of each decision step) to feature dimension. isOptional: true parameterType: NUMBER_DOUBLE gamma_focal_loss: defaultValue: 2.0 - description: 'Gamma value (modulating factor) for - - focal loss for focal loss. Only used for classification.' + description: Gamma value (modulating factor) for focal loss for focal loss. + Only used for classification. isOptional: true parameterType: NUMBER_DOUBLE gradient_thresh: @@ -2695,16 +2531,14 @@ components: parameterType: NUMBER_DOUBLE large_category_dim: defaultValue: 1.0 - description: 'Embedding dimension for categorical - - feature with large number of categories.' + description: Embedding dimension for categorical feature with large number + of categories. isOptional: true parameterType: NUMBER_INTEGER large_category_thresh: defaultValue: 300.0 - description: 'Threshold for number of categories - - to apply large_category_dim embedding dimension to.' + description: Threshold for number of categories to apply large_category_dim + embedding dimension to. isOptional: true parameterType: NUMBER_INTEGER learning_rate: @@ -2715,13 +2549,9 @@ components: parameterType: STRING loss_function_type: defaultValue: default - description: 'Loss function type. Loss function in - - classification [cross_entropy, weighted_cross_entropy, focal_loss], - - default is cross_entropy. Loss function in regression: [rmse, mae, mse], - - default is mse.' + description: 'Loss function type. Loss function in classification [cross_entropy, + weighted_cross_entropy, focal_loss], default is cross_entropy. Loss function + in regression: [rmse, mae, mse], default is mse.' isOptional: true parameterType: STRING max_steps: @@ -2731,20 +2561,14 @@ components: parameterType: NUMBER_INTEGER max_train_secs: defaultValue: -1.0 - description: 'Amount of time in seconds to run the - - trainer for.' + description: Amount of time in seconds to run the trainer for. isOptional: true parameterType: NUMBER_INTEGER measurement_selection_type: defaultValue: BEST_MEASUREMENT - description: 'Which measurement to use - - if/when the service automatically selects the final measurement from - - previously reported intermediate measurements. One of "BEST_MEASUREMENT" - - or "LAST_MEASUREMENT".' + description: Which measurement to use if/when the service automatically + selects the final measurement from previously reported intermediate measurements. + One of "BEST_MEASUREMENT" or "LAST_MEASUREMENT". isOptional: true parameterType: STRING num_decision_steps: @@ -2754,50 +2578,35 @@ components: parameterType: NUMBER_INTEGER num_transformer_layers: defaultValue: 4.0 - description: 'The number of transformer layers - - for each decision step. used only at one decision step and as it - - increases, more flexibility is provided to use a feature at multiple - - decision steps.' + description: The number of transformer layers for each decision step. used + only at one decision step and as it increases, more flexibility is provided + to use a feature at multiple decision steps. isOptional: true parameterType: NUMBER_INTEGER num_transformer_layers_ratio: defaultValue: 0.25 - description: 'The ratio of shared - - transformer layer to transformer layers.' + description: The ratio of shared transformer layer to transformer layers. isOptional: true parameterType: NUMBER_DOUBLE optimization_metric: defaultValue: '' - description: 'Optimization metric used for - - `measurement_selection_type`. Default is "rmse" for regression and "auc" - - for classification.' + description: Optimization metric used for `measurement_selection_type`. + Default is "rmse" for regression and "auc" for classification. isOptional: true parameterType: STRING prediction_type: - description: 'The type of prediction the model is to - - produce. "classification" or "regression".' + description: The type of prediction the model is to produce. "classification" + or "regression". parameterType: STRING project: description: The GCP project that runs the pipeline components. parameterType: STRING relaxation_factor: defaultValue: 1.5 - description: 'Relaxation factor that promotes the - - reuse of each feature at different decision steps. When it is 1, a - - feature is enforced to be used only at one decision step and as it - - increases, more flexibility is provided to use a feature at multiple - - decision steps.' + description: Relaxation factor that promotes the reuse of each feature at + different decision steps. When it is 1, a feature is enforced to be used + only at one decision step and as it increases, more flexibility is provided + to use a feature at multiple decision steps. isOptional: true parameterType: NUMBER_DOUBLE root_dir: @@ -2810,9 +2619,8 @@ components: parameterType: NUMBER_INTEGER sparsity_loss_weight: defaultValue: 1.0e-05 - description: 'Weight of the loss for sparsity - - regularization (increasing it will yield more sparse feature selection).' + description: Weight of the loss for sparsity regularization (increasing + it will yield more sparse feature selection). isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -2828,11 +2636,8 @@ components: training_machine_spec: defaultValue: machine_type: c2-standard-16 - description: 'The training machine - - spec. See https://cloud.google.com/compute/docs/machine-types for - - options.' + description: The training machine spec. See https://cloud.google.com/compute/docs/machine-types + for options. isOptional: true parameterType: STRUCT weight_column: @@ -2842,9 +2647,7 @@ components: parameterType: STRING yeo_johnson_transform: defaultValue: true - description: 'Enables trainable Yeo-Johnson - - power transform.' + description: Enables trainable Yeo-Johnson power transform. isOptional: true parameterType: BOOLEAN outputDefinitions: @@ -2867,16 +2670,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -2884,9 +2683,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -2896,19 +2693,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -2923,76 +2713,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -3002,33 +2777,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -3038,45 +2804,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -3113,7 +2870,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -3128,7 +2885,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -3244,8 +3001,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -3262,7 +3019,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -3569,7 +3326,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -3615,7 +3372,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-tabnet-trainer: container: args: @@ -3633,11 +3390,11 @@ deploymentSpec: "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\":\"", "1", "\", \"machine_spec\": ", "{{$.inputs.parameters[''training_machine_spec'']}}", ", \"disk_spec\": ", "{{$.inputs.parameters[''training_disk_spec'']}}", - ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20231029_0125", + ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/tabnet-training:20240119_0125", "\", \"args\": [\"--target_column=", "{{$.inputs.parameters[''target_column'']}}", "\", \"--weight_column=", "{{$.inputs.parameters[''weight_column'']}}", "\", \"--model_type=", "{{$.inputs.parameters[''prediction_type'']}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--metadata_path=", "{{$.inputs.artifacts[''metadata''].uri}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", @@ -3724,7 +3481,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 pipelineInfo: description: 'Train a model using the Tabular Workflow for TabNet pipelines. diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job.py index 4f1b8f7de9..c08e3bf0c1 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job.py @@ -158,7 +158,7 @@ def wide_and_deep_hyperparameter_tuning_job( ', "disk_spec": ', training_disk_spec, ', "container_spec": {"image_uri":"', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20240119_0125', '", "args": ["--target_column=', target_column, '", "--weight_column=', @@ -166,7 +166,7 @@ def wide_and_deep_hyperparameter_tuning_job( '", "--model_type=', prediction_type, '", "--prediction_docker_uri=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125', '", "--prediction_docker_uri_artifact_path=', prediction_docker_uri_output, '", "--baseline_path=', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job_pipeline.yaml index 10ebb50a76..f2945d427b 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_hyperparameter_tuning_job_pipeline.yaml @@ -83,10 +83,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-infra-validator: executorLabel: exec-automl-tabular-infra-validator @@ -96,9 +94,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-bool-identity: executorLabel: exec-bool-identity inputDefinitions: @@ -788,159 +784,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -950,24 +912,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -983,9 +943,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -1000,17 +958,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -1060,18 +1012,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -1081,9 +1028,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -1116,67 +1061,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -1186,9 +1110,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -1199,25 +1122,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -1241,264 +1159,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -1532,11 +1398,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -1557,36 +1421,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-get-best-hyperparameter-tuning-job-trial: executorLabel: exec-get-best-hyperparameter-tuning-job-trial @@ -2407,16 +2263,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -2424,9 +2276,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -2436,19 +2286,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -2463,76 +2306,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -2542,33 +2370,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -2578,45 +2397,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -2672,16 +2482,13 @@ components: parameters: cache_data: defaultValue: auto - description: 'Whether to cache data or not. If set to - - ''auto'', caching is determined based on the dataset size.' + description: Whether to cache data or not. If set to 'auto', caching is + determined based on the dataset size. isOptional: true parameterType: STRING enable_profiler: defaultValue: false - description: 'Enables profiling and saves a trace - - during evaluation.' + description: Enables profiling and saves a trace during evaluation. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -2691,20 +2498,14 @@ components: parameterType: STRING eval_frequency_secs: defaultValue: 600.0 - description: 'Frequency at which evaluation and - - checkpointing will take place.' + description: Frequency at which evaluation and checkpointing will take place. isOptional: true parameterType: NUMBER_INTEGER eval_steps: defaultValue: 0.0 - description: 'Number of steps to run evaluation for. If not - - specified or negative, it means run evaluation on the whole validation - - dataset. If set to 0, it means run evaluation for a fixed number of - - samples.' + description: Number of steps to run evaluation for. If not specified or + negative, it means run evaluation on the whole validation dataset. If + set to 0, it means run evaluation for a fixed number of samples. isOptional: true parameterType: NUMBER_INTEGER location: @@ -2712,26 +2513,20 @@ components: parameterType: STRING max_failed_trial_count: defaultValue: 0.0 - description: 'The number of failed trials that - - need to be seen before failing the HyperparameterTuningJob. If set to - 0, - - Vertex AI decides how many trials must fail before the whole job fails.' + description: The number of failed trials that need to be seen before failing + the HyperparameterTuningJob. If set to 0, Vertex AI decides how many trials + must fail before the whole job fails. isOptional: true parameterType: NUMBER_INTEGER max_trial_count: description: The desired total number of trials. parameterType: NUMBER_INTEGER parallel_trial_count: - description: 'The desired number of trials to run - - in parallel.' + description: The desired number of trials to run in parallel. parameterType: NUMBER_INTEGER prediction_type: - description: 'The type of prediction the model is to - - produce. "classification" or "regression".' + description: The type of prediction the model is to produce. "classification" + or "regression". parameterType: STRING project: description: The GCP project that runs the pipeline components. @@ -2746,45 +2541,30 @@ components: parameterType: NUMBER_INTEGER study_spec_algorithm: defaultValue: ALGORITHM_UNSPECIFIED - description: 'The search algorithm specified for - - the study. One of ''ALGORITHM_UNSPECIFIED'', ''GRID_SEARCH'', or - - ''RANDOM_SEARCH''.' + description: The search algorithm specified for the study. One of 'ALGORITHM_UNSPECIFIED', + 'GRID_SEARCH', or 'RANDOM_SEARCH'. isOptional: true parameterType: STRING study_spec_measurement_selection_type: defaultValue: BEST_MEASUREMENT - description: 'Which measurement - - to use if/when the service automatically selects the final measurement - - from previously reported intermediate measurements. One of - - "BEST_MEASUREMENT" or "LAST_MEASUREMENT".' + description: Which measurement to use if/when the service automatically + selects the final measurement from previously reported intermediate measurements. + One of "BEST_MEASUREMENT" or "LAST_MEASUREMENT". isOptional: true parameterType: STRING study_spec_metric_goal: - description: 'Optimization goal of the metric, - - possible values: "MAXIMIZE", "MINIMIZE".' + description: 'Optimization goal of the metric, possible values: "MAXIMIZE", + "MINIMIZE".' parameterType: STRING study_spec_metric_id: - description: 'Metric to optimize, , possible - - values: [ ''loss'', ''average_loss'', ''rmse'', ''mae'', ''mql'', ''accuracy'', - ''auc'', ''precision'', ''recall''].' + description: 'Metric to optimize, possible values: [ ''loss'', ''average_loss'', + ''rmse'', ''mae'', ''mql'', ''accuracy'', ''auc'', ''precision'', ''recall''].' parameterType: STRING study_spec_parameters_override: - description: 'List of dictionaries - - representing parameters to optimize. The dictionary key is the - - parameter_id, which is passed to training job as a command line - - argument, and the dictionary value is the parameter specification of the - - metric.' + description: List of dictionaries representing parameters to optimize. The + dictionary key is the parameter_id, which is passed to training job as + a command line argument, and the dictionary value is the parameter specification + of the metric. parameterType: LIST target_column: description: The target column name. @@ -2799,11 +2579,8 @@ components: training_machine_spec: defaultValue: machine_type: c2-standard-16 - description: 'The training machine - - spec. See https://cloud.google.com/compute/docs/machine-types for - - options.' + description: The training machine spec. See https://cloud.google.com/compute/docs/machine-types + for options. isOptional: true parameterType: STRUCT weight_column: @@ -2850,7 +2627,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -2865,7 +2642,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -2981,8 +2758,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -2999,7 +2776,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -3073,7 +2850,7 @@ deploymentSpec: \ = {\n 'instanceSchemaUri': instance_schema_uri,\n 'predictionSchemaUri':\ \ prediction_schema_uri,\n }\n unmanaged_container_model.uri = os.path.join(\n\ \ trials_dir, 'trial_{}'.format(best_trial['id']), 'model'\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-wide-and-deep-study-spec-parameters: container: args: @@ -3413,7 +3190,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -3459,7 +3236,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -3504,7 +3281,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 exec-wide-and-deep-hyperparameter-tuning-job: container: args: @@ -3532,11 +3309,11 @@ deploymentSpec: ", \"trial_job_spec\": {\"worker_pool_specs\": [{\"replica_count\":\"", "1", "\", \"machine_spec\": ", "{{$.inputs.parameters[''training_machine_spec'']}}", ", \"disk_spec\": ", "{{$.inputs.parameters[''training_disk_spec'']}}", - ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20231029_0125", + ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20240119_0125", "\", \"args\": [\"--target_column=", "{{$.inputs.parameters[''target_column'']}}", "\", \"--weight_column=", "{{$.inputs.parameters[''weight_column'']}}", "\", \"--model_type=", "{{$.inputs.parameters[''prediction_type'']}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--prediction_docker_uri_artifact_path=", "{{$.outputs.parameters[''prediction_docker_uri_output''].output_file}}", "\", \"--baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--metadata_path=", "{{$.inputs.artifacts[''metadata''].uri}}", "\", diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer.py b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer.py index dfbb9090cd..dad48cd27f 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer.py @@ -161,7 +161,7 @@ def wide_and_deep_trainer( ', "disk_spec": ', training_disk_spec, ', "container_spec": {"image_uri":"', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20240119_0125', '", "args": ["--target_column=', target_column, '", "--weight_column=', @@ -169,7 +169,7 @@ def wide_and_deep_trainer( '", "--model_type=', prediction_type, '", "--prediction_docker_uri=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125', '", "--baseline_path=', instance_baseline.uri, '", "--metadata_path=', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer_pipeline.yaml index b6923814f8..a8a993ac59 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/wide_and_deep_trainer_pipeline.yaml @@ -100,10 +100,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-infra-validator: executorLabel: exec-automl-tabular-infra-validator @@ -113,9 +111,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-bool-identity: executorLabel: exec-bool-identity inputDefinitions: @@ -818,159 +814,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -980,24 +942,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -1013,9 +973,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -1030,17 +988,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -1090,18 +1042,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -1111,9 +1058,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -1146,67 +1091,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -1216,9 +1140,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -1229,25 +1152,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -1271,264 +1189,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -1562,11 +1428,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -1587,36 +1451,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-model-batch-predict: executorLabel: exec-model-batch-predict @@ -2377,16 +2233,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -2394,9 +2246,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -2406,19 +2256,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -2433,76 +2276,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -2512,33 +2340,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -2548,45 +2367,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -2657,9 +2467,8 @@ components: parameterType: NUMBER_DOUBLE cache_data: defaultValue: auto - description: 'Whether to cache data or not. If set to - - ''auto'', caching is determined based on the dataset size.' + description: Whether to cache data or not. If set to 'auto', caching is + determined based on the dataset size. isOptional: true parameterType: STRING dnn_beta_1: @@ -2674,60 +2483,44 @@ components: parameterType: NUMBER_DOUBLE dnn_dropout: defaultValue: 0.0 - description: 'The probability we will drop out a given - - coordinate.' + description: The probability we will drop out a given coordinate. isOptional: true parameterType: NUMBER_DOUBLE dnn_l1_regularization_strength: defaultValue: 0.0 - description: 'L1 regularization - - strength for dnn_optimizer_type="ftrl".' + description: L1 regularization strength for dnn_optimizer_type="ftrl". isOptional: true parameterType: NUMBER_DOUBLE dnn_l2_regularization_strength: defaultValue: 0.0 - description: 'L2 regularization - - strength for dnn_optimizer_type="ftrl".' + description: L2 regularization strength for dnn_optimizer_type="ftrl". isOptional: true parameterType: NUMBER_DOUBLE dnn_l2_shrinkage_regularization_strength: defaultValue: 0.0 - description: 'L2 shrinkage - - regularization strength for dnn_optimizer_type="ftrl".' + description: L2 shrinkage regularization strength for dnn_optimizer_type="ftrl". isOptional: true parameterType: NUMBER_DOUBLE dnn_learning_rate: - description: 'The learning rate for training the - - deep part of the model.' + description: The learning rate for training the deep part of the model. parameterType: NUMBER_DOUBLE dnn_optimizer_type: defaultValue: ftrl - description: 'The type of optimizer to use for the - - deep part of the model. Choices are "adam", "ftrl" and "sgd". for the - - Adam, FTRL, and Gradient Descent Optimizers, respectively.' + description: The type of optimizer to use for the deep part of the model. + Choices are "adam", "ftrl" and "sgd". for the Adam, FTRL, and Gradient + Descent Optimizers, respectively. isOptional: true parameterType: STRING embed_categories: defaultValue: true - description: 'If set to true, the categorical columns - - will be used embedded and used in the deep part of the model. Embedding - - size is the square root of the column cardinality.' + description: If set to true, the categorical columns will be used embedded + and used in the deep part of the model. Embedding size is the square root + of the column cardinality. isOptional: true parameterType: BOOLEAN enable_profiler: defaultValue: false - description: 'Enables profiling and saves a trace - - during evaluation.' + description: Enables profiling and saves a trace during evaluation. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -2737,48 +2530,35 @@ components: parameterType: STRING eval_frequency_secs: defaultValue: 600.0 - description: 'Frequency at which evaluation and - - checkpointing will take place.' + description: Frequency at which evaluation and checkpointing will take place. isOptional: true parameterType: NUMBER_INTEGER eval_steps: defaultValue: 0.0 - description: 'Number of steps to run evaluation for. If not - - specified or negative, it means run evaluation on the whole validation - - dataset. If set to 0, it means run evaluation for a fixed number of - - samples.' + description: Number of steps to run evaluation for. If not specified or + negative, it means run evaluation on the whole validation dataset. If + set to 0, it means run evaluation for a fixed number of samples. isOptional: true parameterType: NUMBER_INTEGER hidden_units: defaultValue: 30,30,30 - description: 'Hidden layer sizes to use for DNN feature - - columns, provided in comma-separated layers.' + description: Hidden layer sizes to use for DNN feature columns, provided + in comma-separated layers. isOptional: true parameterType: STRING l1_regularization_strength: defaultValue: 0.0 - description: 'L1 regularization strength - - for optimizer_type="ftrl".' + description: L1 regularization strength for optimizer_type="ftrl". isOptional: true parameterType: NUMBER_DOUBLE l2_regularization_strength: defaultValue: 0.0 - description: 'L2 regularization strength - - for optimizer_type="ftrl"' + description: L2 regularization strength for optimizer_type="ftrl" isOptional: true parameterType: NUMBER_DOUBLE l2_shrinkage_regularization_strength: defaultValue: 0.0 - description: 'L2 shrinkage - - regularization strength for optimizer_type="ftrl".' + description: L2 shrinkage regularization strength for optimizer_type="ftrl". isOptional: true parameterType: NUMBER_DOUBLE learning_rate: @@ -2794,44 +2574,31 @@ components: parameterType: NUMBER_INTEGER max_train_secs: defaultValue: -1.0 - description: 'Amount of time in seconds to run the - - trainer for.' + description: Amount of time in seconds to run the trainer for. isOptional: true parameterType: NUMBER_INTEGER measurement_selection_type: defaultValue: BEST_MEASUREMENT - description: 'Which measurement to use - - if/when the service automatically selects the final measurement from - - previously reported intermediate measurements. One of "BEST_MEASUREMENT" - - or "LAST_MEASUREMENT".' + description: Which measurement to use if/when the service automatically + selects the final measurement from previously reported intermediate measurements. + One of "BEST_MEASUREMENT" or "LAST_MEASUREMENT". isOptional: true parameterType: STRING optimization_metric: defaultValue: '' - description: 'Optimization metric used for - - `measurement_selection_type`. Default is "rmse" for regression and "auc" - - for classification.' + description: Optimization metric used for `measurement_selection_type`. + Default is "rmse" for regression and "auc" for classification. isOptional: true parameterType: STRING optimizer_type: defaultValue: adam - description: 'The type of optimizer to use. Choices are - - "adam", "ftrl" and "sgd" for the Adam, FTRL, and Gradient Descent - - Optimizers, respectively.' + description: The type of optimizer to use. Choices are "adam", "ftrl" and + "sgd" for the Adam, FTRL, and Gradient Descent Optimizers, respectively. isOptional: true parameterType: STRING prediction_type: - description: 'The type of prediction the model is to - - produce. "classification" or "regression".' + description: The type of prediction the model is to produce. "classification" + or "regression". parameterType: STRING project: description: The GCP project that runs the pipeline components. @@ -2857,18 +2624,14 @@ components: training_machine_spec: defaultValue: machine_type: c2-standard-16 - description: 'The training machine - - spec. See https://cloud.google.com/compute/docs/machine-types for - - options.' + description: The training machine spec. See https://cloud.google.com/compute/docs/machine-types + for options. isOptional: true parameterType: STRUCT use_wide: defaultValue: true - description: 'If set to true, the categorical columns will be - - used in the wide part of the DNN model.' + description: If set to true, the categorical columns will be used in the + wide part of the DNN model. isOptional: true parameterType: BOOLEAN weight_column: @@ -2906,7 +2669,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -2921,7 +2684,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -3037,8 +2800,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -3055,7 +2818,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -3314,7 +3077,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -3360,7 +3123,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -3405,7 +3168,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 exec-wide-and-deep-trainer: container: args: @@ -3423,11 +3186,11 @@ deploymentSpec: "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\":\"", "1", "\", \"machine_spec\": ", "{{$.inputs.parameters[''training_machine_spec'']}}", ", \"disk_spec\": ", "{{$.inputs.parameters[''training_disk_spec'']}}", - ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20231029_0125", + ", \"container_spec\": {\"image_uri\":\"", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/wide-and-deep-training:20240119_0125", "\", \"args\": [\"--target_column=", "{{$.inputs.parameters[''target_column'']}}", "\", \"--weight_column=", "{{$.inputs.parameters[''weight_column'']}}", "\", \"--model_type=", "{{$.inputs.parameters[''prediction_type'']}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--metadata_path=", "{{$.inputs.artifacts[''metadata''].uri}}", "\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_hyperparameter_tuning_job_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_hyperparameter_tuning_job_pipeline.yaml index 4df9f4bae9..81f211fdc4 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_hyperparameter_tuning_job_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_hyperparameter_tuning_job_pipeline.yaml @@ -83,10 +83,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-bool-identity: executorLabel: exec-bool-identity @@ -774,159 +772,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -936,24 +900,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -969,9 +931,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -986,17 +946,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -1046,18 +1000,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -1067,9 +1016,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -1102,67 +1049,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -1172,9 +1098,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -1185,25 +1110,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -1227,264 +1147,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -1518,11 +1386,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -1543,36 +1409,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-generate-xgboost-hyperparameter-tuning-worker-pool-specs: executorLabel: exec-generate-xgboost-hyperparameter-tuning-worker-pool-specs @@ -2493,16 +2351,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -2510,9 +2364,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -2522,19 +2374,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -2549,76 +2394,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -2628,33 +2458,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -2664,45 +2485,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -2735,67 +2547,46 @@ components: parameterType: STRING max_failed_trial_count: defaultValue: 0.0 - description: 'The number of failed trials that - - need to be seen before failing the HyperparameterTuningJob. If set to - 0, - - Vertex AI decides how many trials must fail before the whole job fails.' + description: The number of failed trials that need to be seen before failing + the HyperparameterTuningJob. If set to 0, Vertex AI decides how many trials + must fail before the whole job fails. isOptional: true parameterType: NUMBER_INTEGER max_trial_count: description: The desired total number of trials. parameterType: NUMBER_INTEGER parallel_trial_count: - description: 'The desired number of trials to run - - in parallel.' + description: The desired number of trials to run in parallel. parameterType: NUMBER_INTEGER project: description: The GCP project that runs the pipeline components. parameterType: STRING study_spec_algorithm: defaultValue: ALGORITHM_UNSPECIFIED - description: 'The search algorithm specified for - - the study. One of ''ALGORITHM_UNSPECIFIED'', ''GRID_SEARCH'', or - - ''RANDOM_SEARCH''.' + description: The search algorithm specified for the study. One of 'ALGORITHM_UNSPECIFIED', + 'GRID_SEARCH', or 'RANDOM_SEARCH'. isOptional: true parameterType: STRING study_spec_measurement_selection_type: defaultValue: BEST_MEASUREMENT - description: 'Which measurement - - to use if/when the service automatically selects the final measurement - - from previously reported intermediate measurements. One of - - "BEST_MEASUREMENT" or "LAST_MEASUREMENT".' + description: Which measurement to use if/when the service automatically + selects the final measurement from previously reported intermediate measurements. + One of "BEST_MEASUREMENT" or "LAST_MEASUREMENT". isOptional: true parameterType: STRING study_spec_metric_goal: - description: 'Optimization goal of the metric, - - possible values: "MAXIMIZE", "MINIMIZE".' + description: 'Optimization goal of the metric, possible values: "MAXIMIZE", + "MINIMIZE".' parameterType: STRING study_spec_metric_id: - description: 'Metric to optimize. For options, - - please look under ''eval_metric'' at - - https://xgboost.readthedocs.io/en/stable/parameter.html#learning-task-parameters.' + description: Metric to optimize. For options, please look under 'eval_metric' + at https://xgboost.readthedocs.io/en/stable/parameter.html#learning-task-parameters. parameterType: STRING study_spec_parameters_override: - description: 'List of dictionaries - - representing parameters to optimize. The dictionary key is the - - parameter_id, which is passed to training job as a command line - - argument, and the dictionary value is the parameter specification of the - - metric.' + description: List of dictionaries representing parameters to optimize. The + dictionary key is the parameter_id, which is passed to training job as + a command line argument, and the dictionary value is the parameter specification + of the metric. parameterType: LIST worker_pool_specs: description: The worker pool specs. @@ -2803,9 +2594,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'Serialized gcp_resources proto tracking the custom training - - job.' + description: Serialized gcp_resources proto tracking the custom training + job. parameterType: STRING deploymentSpec: executors: @@ -2825,7 +2615,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -2947,8 +2737,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -2965,7 +2755,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -3035,7 +2825,7 @@ deploymentSpec: \ return re.sub(r'^/gcs/', r'gs://', path)\n\n master_worker_pool_spec\ \ = {\n 'replica_count': 1,\n 'machine_spec': {\n 'machine_type':\ \ machine_type,\n },\n 'container_spec': {\n 'image_uri':\ - \ 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/xgboost-training:20231029_0125',\n\ + \ 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/xgboost-training:20240119_0125',\n\ \ 'args': [\n f'--job_dir={get_gcs_path(job_dir)}',\n\ \ f'--instance_schema_path={get_gcs_path(instance_schema_uri)}',\n\ \ f'--prediction_schema_path={get_gcs_path(prediction_schema_uri)}',\n\ @@ -3048,7 +2838,7 @@ deploymentSpec: \ f'--baseline_path={get_gcs_path(instance_baseline)}',\n \ \ f'--eval_metric={eval_metric}',\n f'--disable_default_eval_metric={disable_default_eval_metric}',\n\ \ f'--seed={seed}',\n f'--seed_per_iteration={seed_per_iteration}',\n\ - \ '--prediction_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/xgboost-prediction-server:20231029_0125',\n\ + \ '--prediction_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/xgboost-prediction-server:20240119_0125',\n\ \ ],\n },\n }\n\n # Add optional arguments if set\n if\ \ weight_column:\n master_worker_pool_spec['container_spec']['args'].append(\n\ \ f'--weight_column={weight_column}'\n )\n\n # Add accelerator_type\ @@ -3138,7 +2928,7 @@ deploymentSpec: \ = {\n 'instanceSchemaUri': instance_schema_uri,\n 'predictionSchemaUri':\ \ prediction_schema_uri,\n }\n unmanaged_container_model.uri = os.path.join(\n\ \ trials_dir, 'trial_{}'.format(best_trial['id']), 'model'\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-prediction-type-for-xgboost: container: args: @@ -3757,7 +3547,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -3803,7 +3593,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -3848,7 +3638,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 exec-xgboost-hyperparameter-tuning-job: container: args: diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_trainer_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_trainer_pipeline.yaml index b097aeeb04..4e7fc3dd3d 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_trainer_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/preview/automl/tabular/xgboost_trainer_pipeline.yaml @@ -112,10 +112,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-bool-identity: executorLabel: exec-bool-identity @@ -874,159 +872,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -1036,24 +1000,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -1069,9 +1031,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -1086,17 +1046,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -1146,18 +1100,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -1167,9 +1116,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -1202,67 +1149,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -1272,9 +1198,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -1285,25 +1210,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -1327,264 +1247,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -1618,11 +1486,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -1643,36 +1509,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-generate-xgboost-trainer-worker-pool-specs: executorLabel: exec-generate-xgboost-trainer-worker-pool-specs @@ -2757,16 +2615,12 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Dataset stats generated by - - feature transform engine.' + description: Dataset stats generated by feature transform engine. instance_schema: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'Schema of input data to the tf_model at - - serving time.' + description: Schema of input data to the tf_model at serving time. training_schema: artifactType: schemaTitle: system.Artifact @@ -2774,9 +2628,7 @@ components: parameters: available_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - available at forecast time.' + description: The names of the columns that are available at forecast time. isOptional: true parameterType: LIST context_window: @@ -2786,19 +2638,12 @@ components: parameterType: NUMBER_INTEGER enable_probabilistic_inference: defaultValue: false - description: 'If probabilistic inference is - - enabled, the model will fit a distribution that captures the uncertainty - - of a prediction. At inference time, the predictive distribution is used - - to make a point prediction that minimizes the optimization objective. - - For example, the mean of a predictive distribution is the point - - prediction that minimizes RMSE loss. If quantiles are specified, then - - the quantiles of the distribution are also returned.' + description: If probabilistic inference is enabled, the model will fit a + distribution that captures the uncertainty of a prediction. At inference + time, the predictive distribution is used to make a point prediction that + minimizes the optimization objective. For example, the mean of a predictive + distribution is the point prediction that minimizes RMSE loss. If quantiles + are specified, then the quantiles of the distribution are also returned. isOptional: true parameterType: BOOLEAN forecast_horizon: @@ -2813,76 +2658,61 @@ components: parameterType: STRING forecasting_transformations: defaultValue: {} - description: 'Dict mapping auto and/or type-resolutions to - - feature columns. The supported types are auto, categorical, numeric, - - text, and timestamp.' + description: Dict mapping auto and/or type-resolutions to feature columns. + The supported types are auto, categorical, numeric, text, and timestamp. isOptional: true parameterType: STRUCT group_columns: - description: 'A list of time series attribute column - - names that define the time series hierarchy.' + description: A list of time series attribute column names that define the + time series hierarchy. isOptional: true parameterType: LIST group_temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over both the horizon and time series in the same - - hierarchy group.' + description: The weight of the loss for predictions aggregated over both + the horizon and time series in the same hierarchy group. isOptional: true parameterType: NUMBER_DOUBLE group_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over time series in the same group.' + description: The weight of the loss for predictions aggregated over time + series in the same group. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE prediction_type: defaultValue: '' - description: 'Model prediction type. One of "classification", - - "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING quantiles: @@ -2892,33 +2722,24 @@ components: parameterType: LIST run_distill: defaultValue: false - description: 'Whether the distillation should be applied to the - - training.' + description: Whether the distillation should be applied to the training. isOptional: true parameterType: BOOLEAN run_evaluation: defaultValue: false - description: 'Whether we are running evaluation in the training - - pipeline.' + description: Whether we are running evaluation in the training pipeline. isOptional: true parameterType: BOOLEAN split_example_counts: - description: 'JSON string of data split example counts for - - train, validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING stage_1_deadline_hours: - description: 'Stage 1 training budget in - - hours.' + description: Stage 1 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE stage_2_deadline_hours: - description: 'Stage 2 training budget in - - hours.' + description: Stage 2 training budget in hours. isOptional: true parameterType: NUMBER_DOUBLE target_column: @@ -2928,45 +2749,36 @@ components: parameterType: STRING temporal_total_weight: defaultValue: 0.0 - description: 'The weight of the loss for - - predictions aggregated over the horizon for a single time series.' + description: The weight of the loss for predictions aggregated over the + horizon for a single time series. isOptional: true parameterType: NUMBER_DOUBLE time_column: defaultValue: '' - description: 'The column that indicates the time. Used by forecasting - - only.' + description: The column that indicates the time. Used by forecasting only. isOptional: true parameterType: STRING time_series_attribute_columns: defaultValue: [] - description: 'The column names of the time series - - attributes.' + description: The column names of the time series attributes. isOptional: true parameterType: LIST time_series_identifier_column: - description: '[Deprecated] The time series identifier - - column. Used by forecasting only. Raises exception if used - - - use the "time_series_identifier_column" field instead.' + description: '[Deprecated] The time series identifier column. Used by forecasting + only. Raises exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING time_series_identifier_columns: defaultValue: [] - description: 'The list of time series identifier columns. - - Used by forecasting only.' + description: The list of time series identifier columns. Used by forecasting + only. isOptional: true parameterType: LIST unavailable_at_forecast_columns: defaultValue: [] - description: 'The names of the columns that are - - not available at forecast time.' + description: The names of the columns that are not available at forecast + time. isOptional: true parameterType: LIST weight_column: @@ -3006,9 +2818,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'Serialized gcp_resources proto tracking the custom training - - job.' + description: Serialized gcp_resources proto tracking the custom training + job. parameterType: STRING deploymentSpec: executors: @@ -3028,7 +2839,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -3150,8 +2961,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -3168,7 +2979,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 30.0 @@ -3294,10 +3105,10 @@ deploymentSpec: \ worker pool specs.\n \"\"\"\n import copy\n import collections\n import\ \ os\n import re\n\n def get_gcs_path(path):\n return re.sub(r'/gcs/',\ \ 'gs://', path)\n\n formatted_job_dir = get_gcs_path(job_dir)\n prediction_docker_uri\ - \ = (\n 'us-docker.pkg.dev/vertex-ai/automl-tabular/xgboost-prediction-server:20231029_0125'\n\ + \ = (\n 'us-docker.pkg.dev/vertex-ai/automl-tabular/xgboost-prediction-server:20240119_0125'\n\ \ )\n master_worker_pool_spec = {\n 'replica_count': 1,\n 'machine_spec':\ \ {\n 'machine_type': machine_type,\n },\n 'container_spec':\ - \ {\n 'image_uri': 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/xgboost-training:20231029_0125',\n\ + \ {\n 'image_uri': 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/xgboost-training:20240119_0125',\n\ \ 'args': [\n f'--job_dir={formatted_job_dir}',\n\ \ f'--target_column={target_column}',\n f'--objective={objective}',\n\ \ f'--training_data_path={get_gcs_path(materialized_train_split)}',\n\ @@ -3591,7 +3402,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-split-materialized-data: container: args: @@ -3637,7 +3448,7 @@ deploymentSpec: \ 'w') as f:\n f.write(file_patterns[0])\n\n with tf.io.gfile.GFile(materialized_eval_split,\ \ 'w') as f:\n f.write(file_patterns[1])\n\n with tf.io.gfile.GFile(materialized_test_split,\ \ 'w') as f:\n f.write(file_patterns[2])\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 exec-training-configurator-and-validator: container: args: @@ -3682,7 +3493,7 @@ deploymentSpec: ["--temporal_total_weight=", "{{$.inputs.parameters[''temporal_total_weight'']}}"]}}}' - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 exec-xgboost-trainer: container: args: diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/llm/__init__.py b/components/google-cloud/google_cloud_pipeline_components/preview/llm/__init__.py index e35f70ef04..d21f29b7fd 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/llm/__init__.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/llm/__init__.py @@ -14,9 +14,11 @@ """Large-language model preview components.""" from google_cloud_pipeline_components.preview.llm.infer.component import infer_pipeline +from google_cloud_pipeline_components.preview.llm.rlaif.component import rlaif_pipeline from google_cloud_pipeline_components.preview.llm.rlhf.component import rlhf_pipeline __all__ = [ 'infer_pipeline', 'rlhf_pipeline', + 'rlaif_pipeline', ] diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlaif/__init__.py b/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlaif/__init__.py new file mode 100644 index 0000000000..c0b27fe241 --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlaif/__init__.py @@ -0,0 +1,13 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlaif/component.py b/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlaif/component.py new file mode 100644 index 0000000000..9c213cf123 --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlaif/component.py @@ -0,0 +1,165 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Defines an RLAIF Kubeflow pipeline. + +This pipeline build preference data from AI feedback, trains a +reward model and performs reinforcement learning. +""" + +from typing import NamedTuple, Optional + +from google_cloud_pipeline_components import _placeholders +from google_cloud_pipeline_components._implementation.llm import online_evaluation_pairwise +from google_cloud_pipeline_components._implementation.llm import preference_data_formatter +from google_cloud_pipeline_components.preview.llm.infer import component as infer +from google_cloud_pipeline_components.preview.llm.rlhf import component as rlhf +import kfp + + +class PipelineOutput(NamedTuple): + model_resource_name: str + endpoint_resource_name: str + + +@kfp.dsl.pipeline( + name='rlaif_pipeline', + description='Performs reinforcement learning from AI feedback.', +) +def rlaif_pipeline( + prompt_dataset: str, + preference_prompt_dataset: str, + large_model_reference: str, + model_display_name: Optional[str] = None, + prompt_sequence_length: int = 512, + target_sequence_length: int = 64, + large_model_a_reference: str = 'text-bison@001', + large_model_b_reference: str = 't5-small', + reward_model_learning_rate_multiplier: float = 1.0, + reinforcement_learning_rate_multiplier: float = 1.0, + reward_model_train_steps: int = 1000, + reinforcement_learning_train_steps: int = 1000, + kl_coeff: float = 0.1, + sampling_strategy: str = 'temperature_sampling', + instruction: Optional[str] = None, + eval_dataset: Optional[str] = None, + project: str = _placeholders.PROJECT_ID_PLACEHOLDER, + location: str = _placeholders.LOCATION_PLACEHOLDER, + tensorboard_resource_id: Optional[str] = None, +) -> PipelineOutput: + # fmt: off + """Performs reinforcement learning from AI feedback. + + At the moment, it only supports summarization task type. + + Args: + prompt_dataset: Cloud storage path to an unlabled JSONL dataset that contains prompts. Text datasets must contain an `input_text` field that contains the prompt. Chat datasets must contain at least 1 message in a `messages` field. Each message must be valid JSON that contains `author` and `content` fields, where valid `author` values are `user` and `assistant` and `content` must be non-empty. Each row may contain multiple messages, but the first and last author must be the `user`. An optional `context` field may be provided for each example in a chat dataset. If provided, the `context` will preprended to the message `content`. The `instruction` serves as the default context. (Useful if most messages use the same system-level context.) Any context provided in the example will override the default value. + preference_prompt_dataset: The prompt dataset used for two models' inferences to build the side by side comparison AI feedback. large_model_reference: Name of the base model. Supported values are `text-bison@001`, `t5-small`, `t5-large`, `t5-xl` and `t5-xxl`. `text-bison@001` and `t5-small` are supported in `us-central1` and `europe-west4`. `t5-large`, `t5-xl` and `t5-xxl` are only supported in `europe-west4`. + model_display_name: Name of the fine-tuned model shown in the Model Registry. If not provided, a default name will be created. + prompt_sequence_length: Maximum tokenized sequence length for input text. Higher values increase memory overhead. This value should be at most 8192. Default value is 512. + target_sequence_length: Maximum tokenized sequence length for target text. Higher values increase memory overhead. This value should be at most 1024. Default value is 64. + large_model_a_reference: Name of a predefined model A for side by side comparison to build the AI feedback dataset. By default, it uses `text-bison@001`. The valid values are `t5-small`, `t5-large`, `t5-xl`, `t5-xxl`, `text-bison@001`, `llama-2-7b`, `llama-2-13b`. + large_model_b_reference: Name of a predefined model B for side by side comparison to build the AI feedback dataset. By default, it uses `t5-small`. The valid values are `t5-small`, `t5-large`, `t5-xl`, `t5-xxl`, `text-bison@001`, `llama-2-7b`, `llama-2-13b`. + reward_model_learning_rate_multiplier: Constant used to adjust the base learning rate used when training a reward model. Multiply by a number > 1 to increase the magnitude of updates applied at each training step or multiply by a number < 1 to decrease the magnitude of updates. Default value is 1.0. + reinforcement_learning_rate_multiplier: Constant used to adjust the base learning rate used during reinforcement learning. Multiply by a number > 1 to increase the magnitude of updates applied at each training step or multiply by a number < 1 to decrease the magnitude of updates. Default value is 1.0. + reward_model_train_steps: Number of steps to use when training a reward model. Default value is 1000. + reinforcement_learning_train_steps: Number of reinforcement learning steps to perform when tuning a base model. Default value is 1000. + kl_coeff: Coefficient for KL penalty. This regularizes the policy model and penalizes if it diverges from its initial distribution. If set to 0, the reference language model is not loaded into memory. Default value is 0.1. + sampling_strategy: The strategy used to candidates for AI feedback. Default is temperature_sampling. Valid values are greedy, temperature_sampling + instruction: This field lets the model know what task it needs to perform. Base models have been trained over a large set of varied instructions. You can give a simple and intuitive description of the task and the model will follow it, e.g., "Classify this movie review as positive or negative" or "Translate this sentence to Danish". Do not specify this if your dataset already prepends the instruction to the inputs field. + eval_dataset: Optional Cloud storage path to an evaluation dataset. If provided, inference will be performed on this dataset after training. The dataset format is jsonl. Each example in the dataset must contain a field `input_text` that contains the prompt. + project: Project used to run custom jobs. If not specified the project used to run the pipeline will be used. + location: Location used to run custom jobs. If not specified the location used to run the pipeline will be used. + tensorboard_resource_id: Optional tensorboard resource id in format `projects/{project_number}/locations/{location}/tensorboards/{tensorboard_id}`. If provided, tensorboard metrics will be uploaded to this location. + + Returns: + model_resource_name: Path to the model uploaded to the Model Registry. This will be an empty string if the model was not deployed. + endpoint_resource_name: Path the Online Prediction Endpoint. This will be an empty string if the model was not deployed. + """ + # fmt: on + id_columns = ['content'] + task = 'summarization@001' + deploy_model = True + + output_prediction_gcs_path_a = infer.infer_pipeline( + large_model_reference=large_model_a_reference, + prompt_dataset=preference_prompt_dataset, + prompt_sequence_length=prompt_sequence_length, + target_sequence_length=target_sequence_length, + sampling_strategy=sampling_strategy, + instruction=instruction, + project=project, + location=location, + ).set_display_name('Inferrer A') + output_prediction_gcs_path_b = infer.infer_pipeline( + large_model_reference=large_model_b_reference, + prompt_dataset=preference_prompt_dataset, + prompt_sequence_length=prompt_sequence_length, + target_sequence_length=target_sequence_length, + sampling_strategy=sampling_strategy, + instruction=instruction, + project=project, + location=location, + ).set_display_name('Inferrer B') + + inference_output_uri = ( + preference_data_formatter.format_preference_input_data( + model_a_inference_dir_uri=output_prediction_gcs_path_a.output, + model_b_inference_dir_uri=output_prediction_gcs_path_b.output, + instruction=instruction, + ) + .set_display_name('Prepare AI Feedback Input') + .output + ) + + autosxs = online_evaluation_pairwise.online_evaluation_pairwise( + inference_output_uri=inference_output_uri, + id_columns=id_columns, + task=task, + ).set_display_name('Build AI Feedback') + + preference_dataset = ( + preference_data_formatter.format_preference_data( + input_uri=autosxs.outputs['judgments_uri'] + ) + .set_display_name('Build Preference Dataset') + .output + ) + + rlhf_outputs = ( + rlhf.rlhf_pipeline( + prompt_dataset=prompt_dataset, + preference_dataset=preference_dataset, + large_model_reference=large_model_reference, + model_display_name=model_display_name, + prompt_sequence_length=prompt_sequence_length, + target_sequence_length=target_sequence_length, + reward_model_train_steps=reward_model_train_steps, + reinforcement_learning_train_steps=reinforcement_learning_train_steps, + reward_model_learning_rate_multiplier=reward_model_learning_rate_multiplier, + reinforcement_learning_rate_multiplier=reinforcement_learning_rate_multiplier, + instruction=instruction, + deploy_model=deploy_model, + eval_dataset=eval_dataset, + kl_coeff=kl_coeff, + project=project, + location=location, + tensorboard_resource_id=tensorboard_resource_id, + ) + .set_display_name('Reinforcement Learning From AI Feedback') + .outputs + ) + return PipelineOutput( + model_resource_name=rlhf_outputs['model_resource_name'], + endpoint_resource_name=rlhf_outputs['endpoint_resource_name'], + ) diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlhf/component.py b/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlhf/component.py index d9f3f0f80b..e3b3448e5b 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlhf/component.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/llm/rlhf/component.py @@ -68,7 +68,7 @@ def rlhf_pipeline( kl_coeff: Coefficient for KL penalty. This regularizes the policy model and penalizes if it diverges from its initial distribution. If set to 0, the reference language model is not loaded into memory. Default value is 0.1. instruction: This field lets the model know what task it needs to perform. Base models have been trained over a large set of varied instructions. You can give a simple and intuitive description of the task and the model will follow it, e.g. "Classify this movie review as positive or negative" or "Translate this sentence to Danish". Do not specify this if your dataset already prepends the instruction to the inputs field. deploy_model: Whether to deploy the model to an endpoint in `us-central1`. Default is True. - eval_dataset: Optional Cloud storage path to an evaluation dataset. If provided, inference will be performed on this dataset after training. The dataset format is jsonl. Each example in the dataset must contain a field `input_text` that contains the prompt. + eval_dataset: Optional Cloud storage path to an evaluation dataset. Note, eval dataset can only be provided for third-party models. If provided, inference will be performed on this dataset after training. The dataset format is jsonl. Each example in the dataset must contain a field `input_text` that contains the prompt. project: Project used to run custom jobs. If not specified the project used to run the pipeline will be used. location: Location used to run custom jobs. If not specified the location used to run the pipeline will be used. tensorboard_resource_id: Optional tensorboard resource id in format `projects/{project_number}/locations/{location}/tensorboards/{tensorboard_id}`. If provided, tensorboard metrics will be uploaded to this location. @@ -78,6 +78,12 @@ def rlhf_pipeline( endpoint_resource_name: Path the Online Prediction Endpoint. This will be an empty string if the model was not deployed. """ # fmt: on + + function_based.validate_rlhf_inputs( + large_model_reference=large_model_reference, + eval_dataset=eval_dataset, + ).set_display_name('Validate Inputs') + reward_model_pipeline = ( reward_model_graph.pipeline( preference_dataset=preference_dataset, @@ -110,22 +116,30 @@ def rlhf_pipeline( tensorboard_resource_id=tensorboard_resource_id, ).set_display_name('Reinforcement Learning') - should_perform_inference = function_based.value_exists( + has_inference_dataset = function_based.value_exists( value=eval_dataset ).set_display_name('Resolve Inference Dataset') with kfp.dsl.Condition( - should_perform_inference.output == True, name='Perform Inference' # pylint: disable=singleton-comparison + has_inference_dataset.output == True, # pylint: disable=singleton-comparison + name='Perform Inference', ): - component.infer_pipeline( - project=project, - location=location, - large_model_reference=large_model_reference, - model_checkpoint=rl_model_pipeline.outputs['output_model_path'], - prompt_dataset=eval_dataset, - prompt_sequence_length=prompt_sequence_length, - target_sequence_length=target_sequence_length, - instruction=instruction, - ) + has_model_checkpoint = function_based.value_exists( + value=rl_model_pipeline.outputs['output_model_path'] + ).set_display_name('Resolve Model Checkpoint') + with kfp.dsl.Condition( + has_model_checkpoint.output == True, # pylint: disable=singleton-comparison + name='Test Model Checkpoint Exists', + ): + component.infer_pipeline( + project=project, + location=location, + large_model_reference=large_model_reference, + model_checkpoint=rl_model_pipeline.outputs['output_model_path'], + prompt_dataset=eval_dataset, + prompt_sequence_length=prompt_sequence_length, + target_sequence_length=target_sequence_length, + instruction=instruction, + ) llm_model_handler = deployment_graph.pipeline( output_adapter_path=rl_model_pipeline.outputs['output_adapter_path'], diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_classification_pipeline.py b/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_classification_pipeline.py index 6d00a33028..765b0fdf62 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_classification_pipeline.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_classification_pipeline.py @@ -35,7 +35,7 @@ def evaluation_llm_classification_pipeline( # pylint: disable=dangerous-default target_field_name: str, batch_predict_gcs_source_uris: List[str], batch_predict_gcs_destination_output_uri: str, - model_name: str = 'publishers/google/models/text-bison@001', + model_name: str = 'publishers/google/models/text-bison@002', evaluation_task: str = 'text-classification', evaluation_class_labels: List[str] = [], input_field_name: str = 'input_text', diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_text_generation_pipeline.py b/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_text_generation_pipeline.py index 0002cdd5e9..497b91d75a 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_text_generation_pipeline.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/evaluation_llm_text_generation_pipeline.py @@ -33,7 +33,7 @@ def evaluation_llm_text_generation_pipeline( # pylint: disable=dangerous-defaul location: str, batch_predict_gcs_source_uris: List[str], batch_predict_gcs_destination_output_uri: str, - model_name: str = 'publishers/google/models/text-bison@001', + model_name: str = 'publishers/google/models/text-bison@002', evaluation_task: str = 'text-generation', input_field_name: str = 'input_text', target_field_name: str = 'output_text', @@ -58,7 +58,18 @@ def evaluation_llm_text_generation_pipeline( # pylint: disable=dangerous-defaul Args: project: Required. The GCP project that runs the pipeline components. location: Required. The GCP region that runs the pipeline components. - batch_predict_gcs_source_uris: Required. Google Cloud Storage URI(-s) to your eval dataset instances data to run batch prediction on. The instances data should also contain the ground truth (target) data, used for evaluation. May contain wildcards. For more information on wildcards, see https://cloud.google.com/storage/docs/gsutil/addlhelp/WildcardNames. For more details about this input config, see https://cloud.google.com/vertex-ai/docs/reference/rest/v1/projects.locations.batchPredictionJobs#InputConfig. + batch_predict_gcs_source_uris: Required. Google Cloud Storage URI(s) to your eval dataset instances data to run batch prediction on. The instances data should also contain the ground truth (target) data, used for evaluation. May contain wildcards. For more information on [wildcards](https://cloud.google.com/storage/docs/gsutil/addlhelp/WildcardNames). For more details about this [input config](https://cloud.google.com/vertex-ai/docs/reference/rest/v1/projects.locations.batchPredictionJobs#InputConfig). The content of gcs source files should be preset to one of the following formats: + 1) Prediction & Evaluation Dataset format, guaranteeing "prompt" and "ground_truth" attributes are included + { + "prompt": "your input/prompt text", + "ground_truth": "your ground truth output text" + } + or + 2) Tuning Dataset format, guaranteeing "input_text" and "output_text" attributes are included. + { + "input_text": "your input/prompt text", + "output_text": "your ground truth output text" + } batch_predict_gcs_destination_output_uri: Required. The Google Cloud Storage location of the directory where the eval pipeline output is to be written to. model_name: The Model name used to run evaluation. Must be a publisher Model or a managed Model sharing the same ancestor location. Starting this job has no impact on any existing deployments of the Model and their resources. evaluation_task: The task that the large language model will be evaluated on. The evaluation component computes a set of metrics relevant to that specific task. Currently supported tasks are: `summarization`, `question-answering`, `text-generation`. @@ -67,6 +78,7 @@ def evaluation_llm_text_generation_pipeline( # pylint: disable=dangerous-defaul batch_predict_instances_format: The format in which instances are given, must be one of the Model's supportedInputStorageFormats. Only "jsonl" is currently supported. For more details about this input config, see https://cloud.google.com/vertex-ai/docs/reference/rest/v1/projects.locations.batchPredictionJobs#InputConfig. batch_predict_predictions_format: The format in which Vertex AI gives the predictions. Must be one of the Model's supportedOutputStorageFormats. Only "jsonl" is currently supported. For more details about this output config, see https://cloud.google.com/vertex-ai/docs/reference/rest/v1/projects.locations.batchPredictionJobs#OutputConfig. batch_predict_model_parameters: A map of parameters that govern the predictions. Some acceptable parameters include: maxOutputTokens, topK, topP, and temperature. + enable_row_based_metrics: Flag of if row based metrics is enabled, default value is false. machine_type: The machine type of this custom job. If not set, defaulted to `e2-highmem-16`. More details: https://cloud.google.com/compute/docs/machine-resource service_account: Sets the default service account for workload run-as account. The service account running the pipeline (https://cloud.google.com/vertex-ai/docs/pipelines/configure-project#service-account) submitting jobs must have act-as permission on this run-as account. If unspecified, the Vertex AI Custom Code Service Agent(https://cloud.google.com/vertex-ai/docs/general/access-control#service-agents) for the CustomJob's project. network: The full name of the Compute Engine network to which the job should be peered. For example, `projects/12345/global/networks/myVPC`. Format is of the form `projects/{project}/global/networks/{network}`. Where `{project}` is a project number, as in `12345`, and `{network}` is a network name, as in `myVPC`. To specify this field, you must have already configured VPC Network Peering for Vertex AI (https://cloud.google.com/vertex-ai/docs/general/vpc-peering). If left unspecified, the job is not peered with any network. @@ -134,21 +146,33 @@ def evaluation_llm_text_generation_pipeline( # pylint: disable=dangerous-defaul encryption_spec_key_name=encryption_spec_key_name, ) - import_evaluation_task = ModelImportEvaluationOp( - metrics=eval_task.outputs['evaluation_metrics'], - row_based_metrics=eval_task.outputs['row_based_metrics'] - if enable_row_based_metrics - else None, - model=get_vertex_model_task.outputs['artifact'], - problem_type=evaluation_task, - dataset_type=batch_predict_predictions_format, - dataset_paths=batch_predict_gcs_source_uris, - display_name=evaluation_display_name, - ) + with dsl.If(enable_row_based_metrics == True): + import_evaluation_task_with_row_based_metrics = ModelImportEvaluationOp( + metrics=eval_task.outputs['evaluation_metrics'], + row_based_metrics=eval_task.outputs['row_based_metrics'], + model=get_vertex_model_task.outputs['artifact'], + problem_type=evaluation_task, + dataset_type=batch_predict_predictions_format, + dataset_paths=batch_predict_gcs_source_uris, + display_name=evaluation_display_name, + ) + with dsl.Else(): + import_evaluation_task = ModelImportEvaluationOp( + metrics=eval_task.outputs['evaluation_metrics'], + model=get_vertex_model_task.outputs['artifact'], + problem_type=evaluation_task, + dataset_type=batch_predict_predictions_format, + dataset_paths=batch_predict_gcs_source_uris, + display_name=evaluation_display_name, + ) - return outputs( - evaluation_metrics=eval_task.outputs['evaluation_metrics'], - evaluation_resource_name=import_evaluation_task.outputs[ + oneof = dsl.OneOf( + import_evaluation_task_with_row_based_metrics.outputs[ 'evaluation_resource_name' ], + import_evaluation_task.outputs['evaluation_resource_name'], + ) + return outputs( + evaluation_metrics=eval_task.outputs['evaluation_metrics'], + evaluation_resource_name=oneof, ) diff --git a/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/model_based_llm_evaluation/autosxs/autosxs_pipeline.py b/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/model_based_llm_evaluation/autosxs/autosxs_pipeline.py index a0a9f7b7a1..00e85b8f87 100644 --- a/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/model_based_llm_evaluation/autosxs/autosxs_pipeline.py +++ b/components/google-cloud/google_cloud_pipeline_components/preview/model_evaluation/model_based_llm_evaluation/autosxs/autosxs_pipeline.py @@ -16,73 +16,12 @@ from typing import Any, Dict, List from google_cloud_pipeline_components import _placeholders -from google_cloud_pipeline_components._implementation.llm import arbiter_preprocess -from google_cloud_pipeline_components._implementation.llm import autosxs_arbiter -from google_cloud_pipeline_components._implementation.llm import autosxs_metrics_computer -from google_cloud_pipeline_components._implementation.llm import function_based -from google_cloud_pipeline_components._implementation.llm import task_preprocess -from google_cloud_pipeline_components.types import artifact_types -from google_cloud_pipeline_components.v1 import batch_predict_job +from google_cloud_pipeline_components._implementation.llm import batch_prediction_pairwise +from google_cloud_pipeline_components._implementation.llm import model_evaluation_text_generation_pairwise +from google_cloud_pipeline_components._implementation.llm import online_evaluation_pairwise from kfp import dsl -# pylint: disable=no-value-for-parameter -@dsl.pipeline( - name='predictions-pipeline', - description='Runs the prediction pipeline for one of the two SxS models.', -) -def _get_predictions( - name: str, - project: str, - location: str, - model: str, - model_parameters: Dict[str, str], - prediction_inputs: List[str], - is_model_inference: bool, -) -> str: - """Makes predictions for a given model.""" - with dsl.If(is_model_inference == True, name='Inference Required'): # pylint: disable=singleton-comparison - get_vertex_model_task = dsl.importer( - artifact_uri=( - f'https://{location}-aiplatform.googleapis.com/v1/{model}' - ), - artifact_class=artifact_types.VertexModel, - metadata={'resourceName': model}, - ).set_display_name('Import Vertex Model Artifact') - - batch_predict_task = batch_predict_job.ModelBatchPredictOp( - project=project, - location=location, - model=get_vertex_model_task.outputs['artifact'], - job_display_name=( - f'autosxs-{name}-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}' - ), - gcs_source_uris=prediction_inputs, - instances_format='jsonl', - predictions_format='jsonl', - gcs_destination_output_uri_prefix=( - f'{dsl.PIPELINE_ROOT_PLACEHOLDER}/{dsl.PIPELINE_TASK_ID_PLACEHOLDER}' - f'/{name}_predictions' - ), - model_parameters=model_parameters, - ) - prediction_uris_from_inference = function_based.get_uri( - artifact=batch_predict_task.outputs['gcs_output_directory'], - is_dir=True, - ) - - with dsl.Else(name='Responses Provided'): # pylint: disable=singleton-comparison - prediction_uris_inference_provided = function_based.get_empty_string() - - prediction_uris = dsl.OneOf( - prediction_uris_from_inference.output, - prediction_uris_inference_provided.output, - ) - - # We can't directly output dsl.OneOf, so we need to use identity. - return function_based.identity(x=prediction_uris).output - - # pylint: disable=dangerous-default-value,g-bare-generic,unused-argument @dsl.pipeline( name='autosxs-template', @@ -113,7 +52,7 @@ def autosxs_pipeline( Args: evaluation_dataset: A BigQuery table or comma-separated list of GCS paths to a JSONL dataset containing evaluation examples. - task: Evaluation task in the form `{task}@{version}`. task can be one of `[summarization, question_answer]`. Version is an integer with 3 digits or "latest". Ex: `summarization@001` or `question_answer@latest`. + task: Evaluation task in the form `{task}@{version}`. task can be one of `[summarization, question_answering]`. Version is an integer with 3 digits or "latest". Ex: `summarization@001` or `question_answering@latest`. id_columns: The columns which distinguish unique evaluation examples. model_a: A fully-qualified model resource name (`projects/{project}/locations/{location}/models/{model}@{version}`) or publisher model resource name (`publishers/{publisher}/models/{model}`). This parameter is optional if Model A responses are specified. model_b: A fully-qualified model resource name (`projects/{project}/locations/{location}/models/{model}@{version}`) or publisher model resource name (`publishers/{publisher}/models/{model}`). This parameter is optional if Model B responses are specified. @@ -125,79 +64,31 @@ def autosxs_pipeline( model_a_parameters: The parameters that govern the predictions from model A, such as temperature or maximum output tokens. model_b_parameters: The parameters that govern the predictions from model B, such as temperature or maximum output tokens. human_preference_column: The column containing ground truth winners for each example. Providing this parameter adds additional metrics for checking the AutoRater alignment with human preferences. - project: Project used to run custom jobs. Default is the same project used to run the pipeline. - location: Location used to run custom jobs. Default is the same location used to run the pipeline. + project: Project used to run custom jobs. This should be the same project used to run the pipeline. + location: Location used to run custom jobs. This should be the same location used to run the pipeline. judgments_format: The format to write judgments to. Can be either `[json, bigquery]`. bigquery_destination_prefix: BigQuery table to write judgments to if the specified format is 'bigquery'. experimental_args: Experimentally released arguments. Subject to change. """ # fmt: on - prediction_inputs_a = task_preprocess.task_preprocess( + responses = batch_prediction_pairwise.batch_prediction_pairwise( + display_name='autosxs-{{$.pipeline_job_uuid}}-{{$.pipeline_task_uuid}}', evaluation_dataset=evaluation_dataset, - task=task, - model_prompt_parameters=model_a_prompt_parameters, - response_column=response_column_a, - human_preference_column=human_preference_column, id_columns=id_columns, - ).set_display_name('Preprocess Model A Inputs') - - prediction_inputs_b = task_preprocess.task_preprocess( - evaluation_dataset=evaluation_dataset, task=task, - model_prompt_parameters=model_b_prompt_parameters, - response_column=response_column_b, - human_preference_column=human_preference_column, - id_columns=id_columns, - ).set_display_name('Preprocess Model B Inputs') - - is_model_a_inference = function_based.get_usage_metric( - metadata=prediction_inputs_a.outputs['metadata'], - key='is_model_inference', - ).set_display_name('Read is_model_a_inference') - - is_model_b_inference = function_based.get_usage_metric( - metadata=prediction_inputs_b.outputs['metadata'], - key='is_model_inference', - ).set_display_name('Read is_model_b_inference') - - inferrer_a = _get_predictions( - name='A', - project=project, - location=location, - model=model_a, - model_parameters=model_a_parameters, - prediction_inputs=prediction_inputs_a.outputs['prediction_inputs'], - is_model_inference=is_model_a_inference.output, - ).set_display_name('Model A Responses') - - inferrer_b = _get_predictions( - name='B', - project=project, - location=location, - model=model_b, - model_parameters=model_b_parameters, - prediction_inputs=prediction_inputs_b.outputs['prediction_inputs'], - is_model_inference=is_model_b_inference.output, - ).set_display_name('Model B Responses') - - arbiter_input_preprocess = arbiter_preprocess.arbiter_preprocess( autorater_prompt_parameters=autorater_prompt_parameters, - evaluation_dataset=evaluation_dataset, - id_columns=id_columns, - prediction_uris_b=inferrer_b.output, - prediction_uris_a=inferrer_a.output, - model_a_prompt_parameters=model_a_prompt_parameters, - model_b_prompt_parameters=model_b_prompt_parameters, - task=task, response_column_a=response_column_a, response_column_b=response_column_b, + model_a=model_a, + model_b=model_b, + model_a_prompt_parameters=model_a_prompt_parameters, + model_b_prompt_parameters=model_b_prompt_parameters, + model_a_parameters=model_a_parameters, + model_b_parameters=model_b_parameters, human_preference_column=human_preference_column, - is_bp_output_a=is_model_a_inference.output, - is_bp_output_b=is_model_b_inference.output, - ).set_display_name('Preprocess Predictions') - - autosxs_arbiter_task = autosxs_arbiter.autosxs_arbiter( - inference_output_uri=arbiter_input_preprocess.outputs[ + ).set_display_name('AutoSxS Batch Prediction') + winners = online_evaluation_pairwise.online_evaluation_pairwise( + inference_output_uri=responses.outputs[ 'preprocessed_evaluation_dataset_uri' ], id_columns=id_columns, @@ -206,14 +97,10 @@ def autosxs_pipeline( judgments_format=judgments_format, bigquery_destination_prefix=bigquery_destination_prefix, experimental_args=experimental_args, - ).set_display_name('AutoSxS Arbiter') - - has_human_preference = function_based.get_usage_metric( - metadata=prediction_inputs_a.outputs['metadata'], - key='has_human_preference_column', - ).set_display_name('Read has_human_preference_column') - - autosxs_metrics_computer.autosxs_metrics_computer( - judgments_dir=autosxs_arbiter_task.outputs['judgments_uri'], - has_human_preference=has_human_preference.output, - ).set_display_name('AutoSxS Metrics') + ).set_display_name('AutoSxS Autorater') + model_evaluation_text_generation_pairwise.model_evaluation_text_generation_pairwise( + judgments_dir=winners.outputs['judgments_uri'], + human_preference_column=human_preference_column, + ).set_display_name( + 'AutoSxS Metrics' + ) diff --git a/components/google-cloud/google_cloud_pipeline_components/proto/gcp_resources_pb2.py b/components/google-cloud/google_cloud_pipeline_components/proto/gcp_resources_pb2.py index a3ae900950..c7f482673a 100644 --- a/components/google-cloud/google_cloud_pipeline_components/proto/gcp_resources_pb2.py +++ b/components/google-cloud/google_cloud_pipeline_components/proto/gcp_resources_pb2.py @@ -3,6 +3,7 @@ # source: gcp_resources.proto """Generated protocol buffer code.""" from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import message as _message from google.protobuf import reflection as _reflection from google.protobuf import symbol_database as _symbol_database @@ -14,223 +15,32 @@ from google.rpc import status_pb2 as google_dot_rpc_dot_status__pb2 -DESCRIPTOR = _descriptor.FileDescriptor( - name='gcp_resources.proto', - package='gcp_launcher', - syntax='proto3', - serialized_options=None, - create_key=_descriptor._internal_create_key, - serialized_pb=( - b'\n\x13gcp_resources.proto\x12\x0cgcp_launcher\x1a\x17google/rpc/status.proto"\xe0\x01\n\x0cGcpResources\x12\x36\n\tresources\x18\x01' - b' \x03(\x0b\x32#.gcp_launcher.GcpResources.Resource\x1a\x97\x01\n\x08Resource\x12\x1a\n\rresource_type\x18\x01' - b' \x01(\tH\x00\x88\x01\x01\x12\x19\n\x0cresource_uri\x18\x02' - b' \x01(\tH\x01\x88\x01\x01\x12!\n\x05\x65rror\x18\x03' - b' \x01(\x0b\x32\x12.google.rpc.Status\x12\x0e\n\x06labels\x18\x04' - b' \x03(\tB\x10\n\x0e_resource_typeB\x0f\n\r_resource_urib\x06proto3' - ), - dependencies=[ - google_dot_rpc_dot_status__pb2.DESCRIPTOR, - ], -) +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x13gcp_resources.proto\x12\x0cgcp_launcher\x1a\x17google/rpc/status.proto\"\xe0\x01\n\x0cGcpResources\x12\x36\n\tresources\x18\x01 \x03(\x0b\x32#.gcp_launcher.GcpResources.Resource\x1a\x97\x01\n\x08Resource\x12\x1a\n\rresource_type\x18\x01 \x01(\tH\x00\x88\x01\x01\x12\x19\n\x0cresource_uri\x18\x02 \x01(\tH\x01\x88\x01\x01\x12!\n\x05\x65rror\x18\x03 \x01(\x0b\x32\x12.google.rpc.Status\x12\x0e\n\x06labels\x18\x04 \x03(\tB\x10\n\x0e_resource_typeB\x0f\n\r_resource_urib\x06proto3') -_GCPRESOURCES_RESOURCE = _descriptor.Descriptor( - name='Resource', - full_name='gcp_launcher.GcpResources.Resource', - filename=None, - file=DESCRIPTOR, - containing_type=None, - create_key=_descriptor._internal_create_key, - fields=[ - _descriptor.FieldDescriptor( - name='resource_type', - full_name='gcp_launcher.GcpResources.Resource.resource_type', - index=0, - number=1, - type=9, - cpp_type=9, - label=1, - has_default_value=False, - default_value=b''.decode('utf-8'), - message_type=None, - enum_type=None, - containing_type=None, - is_extension=False, - extension_scope=None, - serialized_options=None, - file=DESCRIPTOR, - create_key=_descriptor._internal_create_key, - ), - _descriptor.FieldDescriptor( - name='resource_uri', - full_name='gcp_launcher.GcpResources.Resource.resource_uri', - index=1, - number=2, - type=9, - cpp_type=9, - label=1, - has_default_value=False, - default_value=b''.decode('utf-8'), - message_type=None, - enum_type=None, - containing_type=None, - is_extension=False, - extension_scope=None, - serialized_options=None, - file=DESCRIPTOR, - create_key=_descriptor._internal_create_key, - ), - _descriptor.FieldDescriptor( - name='error', - full_name='gcp_launcher.GcpResources.Resource.error', - index=2, - number=3, - type=11, - cpp_type=10, - label=1, - has_default_value=False, - default_value=None, - message_type=None, - enum_type=None, - containing_type=None, - is_extension=False, - extension_scope=None, - serialized_options=None, - file=DESCRIPTOR, - create_key=_descriptor._internal_create_key, - ), - _descriptor.FieldDescriptor( - name='labels', - full_name='gcp_launcher.GcpResources.Resource.labels', - index=3, - number=4, - type=9, - cpp_type=9, - label=3, - has_default_value=False, - default_value=[], - message_type=None, - enum_type=None, - containing_type=None, - is_extension=False, - extension_scope=None, - serialized_options=None, - file=DESCRIPTOR, - create_key=_descriptor._internal_create_key, - ), - ], - extensions=[], - nested_types=[], - enum_types=[], - serialized_options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='_resource_type', - full_name='gcp_launcher.GcpResources.Resource._resource_type', - index=0, - containing_type=None, - create_key=_descriptor._internal_create_key, - fields=[], - ), - _descriptor.OneofDescriptor( - name='_resource_uri', - full_name='gcp_launcher.GcpResources.Resource._resource_uri', - index=1, - containing_type=None, - create_key=_descriptor._internal_create_key, - fields=[], - ), - ], - serialized_start=136, - serialized_end=287, -) -_GCPRESOURCES = _descriptor.Descriptor( - name='GcpResources', - full_name='gcp_launcher.GcpResources', - filename=None, - file=DESCRIPTOR, - containing_type=None, - create_key=_descriptor._internal_create_key, - fields=[ - _descriptor.FieldDescriptor( - name='resources', - full_name='gcp_launcher.GcpResources.resources', - index=0, - number=1, - type=11, - cpp_type=10, - label=3, - has_default_value=False, - default_value=[], - message_type=None, - enum_type=None, - containing_type=None, - is_extension=False, - extension_scope=None, - serialized_options=None, - file=DESCRIPTOR, - create_key=_descriptor._internal_create_key, - ), - ], - extensions=[], - nested_types=[ - _GCPRESOURCES_RESOURCE, - ], - enum_types=[], - serialized_options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[], - serialized_start=63, - serialized_end=287, -) +_GCPRESOURCES = DESCRIPTOR.message_types_by_name['GcpResources'] +_GCPRESOURCES_RESOURCE = _GCPRESOURCES.nested_types_by_name['Resource'] +GcpResources = _reflection.GeneratedProtocolMessageType('GcpResources', (_message.Message,), { -# pytype: disable=module-attr -_GCPRESOURCES_RESOURCE.fields_by_name['error'].message_type = ( - google_dot_rpc_dot_status__pb2._STATUS -) -_GCPRESOURCES_RESOURCE.containing_type = _GCPRESOURCES -_GCPRESOURCES_RESOURCE.oneofs_by_name['_resource_type'].fields.append( - _GCPRESOURCES_RESOURCE.fields_by_name['resource_type'] -) -_GCPRESOURCES_RESOURCE.fields_by_name['resource_type'].containing_oneof = ( - _GCPRESOURCES_RESOURCE.oneofs_by_name['_resource_type'] -) -_GCPRESOURCES_RESOURCE.oneofs_by_name['_resource_uri'].fields.append( - _GCPRESOURCES_RESOURCE.fields_by_name['resource_uri'] -) -_GCPRESOURCES_RESOURCE.fields_by_name['resource_uri'].containing_oneof = ( - _GCPRESOURCES_RESOURCE.oneofs_by_name['_resource_uri'] -) -_GCPRESOURCES.fields_by_name['resources'].message_type = _GCPRESOURCES_RESOURCE -DESCRIPTOR.message_types_by_name['GcpResources'] = _GCPRESOURCES -_sym_db.RegisterFileDescriptor(DESCRIPTOR) - -GcpResources = _reflection.GeneratedProtocolMessageType( - 'GcpResources', - (_message.Message,), - { - 'Resource': _reflection.GeneratedProtocolMessageType( - 'Resource', - (_message.Message,), - { - 'DESCRIPTOR': _GCPRESOURCES_RESOURCE, - '__module__': 'gcp_resources_pb2', - # @@protoc_insertion_point(class_scope:gcp_launcher.GcpResources.Resource) - }, - ), - 'DESCRIPTOR': _GCPRESOURCES, - '__module__': 'gcp_resources_pb2', - # @@protoc_insertion_point(class_scope:gcp_launcher.GcpResources) - }, -) + 'Resource' : _reflection.GeneratedProtocolMessageType('Resource', (_message.Message,), { + 'DESCRIPTOR' : _GCPRESOURCES_RESOURCE, + '__module__' : 'gcp_resources_pb2' + # @@protoc_insertion_point(class_scope:gcp_launcher.GcpResources.Resource) + }) + , + 'DESCRIPTOR' : _GCPRESOURCES, + '__module__' : 'gcp_resources_pb2' + # @@protoc_insertion_point(class_scope:gcp_launcher.GcpResources) + }) _sym_db.RegisterMessage(GcpResources) _sym_db.RegisterMessage(GcpResources.Resource) +if _descriptor._USE_C_DESCRIPTORS == False: + DESCRIPTOR._options = None + _GCPRESOURCES._serialized_start=63 + _GCPRESOURCES._serialized_end=287 + _GCPRESOURCES_RESOURCE._serialized_start=136 + _GCPRESOURCES_RESOURCE._serialized_end=287 # @@protoc_insertion_point(module_scope) diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_predict_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_predict_pipeline.yaml index ed4ec361a5..472125a04b 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_predict_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_predict_pipeline.yaml @@ -658,7 +658,7 @@ deploymentSpec: \ = client.create_dataset(dataset=dataset, exists_ok=exists_ok)\n return\ \ collections.namedtuple('Outputs', ['project_id', 'dataset_id'])(\n \ \ ref.project, ref.dataset_id)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-create-dataset-2: container: args: @@ -693,7 +693,7 @@ deploymentSpec: \ = client.create_dataset(dataset=dataset, exists_ok=exists_ok)\n return\ \ collections.namedtuple('Outputs', ['project_id', 'dataset_id'])(\n \ \ ref.project, ref.dataset_id)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-delete-dataset-with-prefix: container: args: @@ -727,7 +727,7 @@ deploymentSpec: \ if dataset.dataset_id.startswith(dataset_prefix):\n client.delete_dataset(\n\ \ dataset=dataset.dataset_id,\n delete_contents=delete_contents)\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-query-job: container: args: @@ -788,7 +788,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-first-valid: container: args: @@ -818,7 +818,7 @@ deploymentSpec: \ import json\n # pylint: enable=g-import-not-at-top,import-outside-toplevel,redefined-outer-name,reimported\n\ \n for value in json.loads(values):\n if value:\n return value\n\ \ raise ValueError('No valid values.')\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-model-metadata: container: args: @@ -857,7 +857,7 @@ deploymentSpec: \ 'forecast_horizon',\n ],\n )(\n options.time_series_timestamp_column,\n\ \ options.time_series_id_column,\n options.time_series_data_column,\n\ \ options.horizon,\n )\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-table-location: container: args: @@ -893,7 +893,7 @@ deploymentSpec: \ if table.startswith('bq://'):\n table = table[len('bq://'):]\n elif\ \ table.startswith('bigquery://'):\n table = table[len('bigquery://'):]\n\ \ return client.get_table(table).location\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-load-table-from-uri: container: args: @@ -934,7 +934,7 @@ deploymentSpec: \ source_format=source_format)\n client.load_table_from_uri(\n source_uris=csv_list,\n\ \ destination=destination,\n project=project,\n location=location,\n\ \ job_config=job_config).result()\n return destination\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-maybe-replace-with-default: container: args: @@ -962,7 +962,7 @@ deploymentSpec: \ *\n\ndef maybe_replace_with_default(value: str, default: str = '') ->\ \ str:\n \"\"\"Replaces string with another value if it is a dash.\"\"\"\ \n return default if not value else value\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-validate-inputs: container: args: @@ -1064,7 +1064,7 @@ deploymentSpec: \ raise ValueError(\n 'Granularity unit should be one of the\ \ following: '\n f'{valid_data_granularity_units}, got: {data_granularity_unit}.')\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 pipelineInfo: description: Forecasts using a BQML ARIMA_PLUS model. name: automl-tabular-bqml-arima-prediction diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_train_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_train_pipeline.yaml index 2d4ed1addd..c786c5c582 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_train_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/bqml_arima_train_pipeline.yaml @@ -2025,159 +2025,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -2187,24 +2153,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -2220,9 +2184,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -2237,17 +2199,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -2297,18 +2253,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -2318,9 +2269,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -2353,67 +2302,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -2423,9 +2351,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -2436,25 +2363,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -2478,264 +2400,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -2769,11 +2639,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -2794,36 +2662,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-for-loop-3: dag: @@ -3539,7 +3399,7 @@ deploymentSpec: \ = client.create_dataset(dataset=dataset, exists_ok=exists_ok)\n return\ \ collections.namedtuple('Outputs', ['project_id', 'dataset_id'])(\n \ \ ref.project, ref.dataset_id)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-create-dataset-2: container: args: @@ -3574,7 +3434,7 @@ deploymentSpec: \ = client.create_dataset(dataset=dataset, exists_ok=exists_ok)\n return\ \ collections.namedtuple('Outputs', ['project_id', 'dataset_id'])(\n \ \ ref.project, ref.dataset_id)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-create-model-job: container: args: @@ -3634,7 +3494,7 @@ deploymentSpec: \ if dataset.dataset_id.startswith(dataset_prefix):\n client.delete_dataset(\n\ \ dataset=dataset.dataset_id,\n delete_contents=delete_contents)\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-list-rows: container: args: @@ -3672,7 +3532,7 @@ deploymentSpec: \ metadata['datasetId'], metadata['tableId']]))\n result = []\n for row\ \ in rows:\n result.append({col: str(value) for col, value in dict(row).items()})\n\ \ return result\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-list-rows-2: container: args: @@ -3710,7 +3570,7 @@ deploymentSpec: \ metadata['datasetId'], metadata['tableId']]))\n result = []\n for row\ \ in rows:\n result.append({col: str(value) for col, value in dict(row).items()})\n\ \ return result\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-query-job: container: args: @@ -3879,7 +3739,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-job-configuration-query-2: container: args: @@ -3913,7 +3773,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-job-configuration-query-3: container: args: @@ -3947,7 +3807,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-job-configuration-query-4: container: args: @@ -3981,7 +3841,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-job-configuration-query-5: container: args: @@ -4015,7 +3875,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-job-configuration-query-6: container: args: @@ -4049,7 +3909,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-serialized-query-parameters: container: args: @@ -4126,7 +3986,7 @@ deploymentSpec: \ 'name': 'start_time',\n 'parameterType': {\n 'type':\ \ 'TIMESTAMP'\n },\n 'parameterValue': {\n 'value': start_time\n\ \ },\n })\n return query_parameters\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-serialized-query-parameters-2: container: args: @@ -4203,7 +4063,7 @@ deploymentSpec: \ 'name': 'start_time',\n 'parameterType': {\n 'type':\ \ 'TIMESTAMP'\n },\n 'parameterValue': {\n 'value': start_time\n\ \ },\n })\n return query_parameters\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-serialized-query-parameters-3: container: args: @@ -4280,7 +4140,7 @@ deploymentSpec: \ 'name': 'start_time',\n 'parameterType': {\n 'type':\ \ 'TIMESTAMP'\n },\n 'parameterValue': {\n 'value': start_time\n\ \ },\n })\n return query_parameters\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-cond: container: args: @@ -4308,7 +4168,7 @@ deploymentSpec: \ *\n\ndef cond(predicate: bool, true_str: str, false_str: str) -> str:\n\ \ \"\"\"Returns true_str if predicate is true, else false_str.\"\"\"\n\ \ return true_str if predicate else false_str\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-create-metrics-artifact: container: args: @@ -4340,7 +4200,7 @@ deploymentSpec: \ 'MAPE': 'meanAbsolutePercentageError',\n }\n metrics = {metric_name_map[k]:\ \ v for k, v in dict(metrics_rows[0]).items()}\n evaluation_metrics.metadata\ \ = metrics\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-feature-transform-engine: container: args: @@ -4425,8 +4285,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -4443,7 +4303,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 exec-get-fte-suffix: container: args: @@ -4477,7 +4337,7 @@ deploymentSpec: \ table.table_id.startswith(fte_table):\n return table.table_id[len(fte_table)\ \ + 1:]\n raise ValueError(\n f'No FTE output tables found in {bigquery_staging_full_dataset_id}.')\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-table-location: container: args: @@ -4513,7 +4373,7 @@ deploymentSpec: \ if table.startswith('bq://'):\n table = table[len('bq://'):]\n elif\ \ table.startswith('bigquery://'):\n table = table[len('bigquery://'):]\n\ \ return client.get_table(table).location\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-value: container: args: @@ -4540,7 +4400,7 @@ deploymentSpec: - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ \ *\n\ndef get_value(d: Dict[str, str], key: str) -> str:\n return d[key]\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-window-query-priority: container: args: @@ -4570,7 +4430,7 @@ deploymentSpec: \ depending on the window number.\"\"\"\n if int(window['window_number'])\ \ <= max_interactive:\n return 'INTERACTIVE'\n else:\n return 'BATCH'\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-maybe-replace-with-default: container: args: @@ -4598,7 +4458,7 @@ deploymentSpec: \ *\n\ndef maybe_replace_with_default(value: str, default: str = '') ->\ \ str:\n \"\"\"Replaces string with another value if it is a dash.\"\"\"\ \n return default if not value else value\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-query-with-retry: container: args: @@ -4652,7 +4512,7 @@ deploymentSpec: \ 'Query failed with %s. Retrying after %d seconds.', e, wait_time)\n\ \ time.sleep(wait_time)\n retry_count += 1\n return destination_uri\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-query-with-retry-2: container: args: @@ -4706,7 +4566,7 @@ deploymentSpec: \ 'Query failed with %s. Retrying after %d seconds.', e, wait_time)\n\ \ time.sleep(wait_time)\n retry_count += 1\n return destination_uri\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-query-with-retry-3: container: args: @@ -4760,7 +4620,7 @@ deploymentSpec: \ 'Query failed with %s. Retrying after %d seconds.', e, wait_time)\n\ \ time.sleep(wait_time)\n retry_count += 1\n return destination_uri\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-table-to-uri: container: args: @@ -4796,7 +4656,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-table-to-uri-2: container: args: @@ -4832,7 +4692,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-validate-inputs: container: args: @@ -4934,7 +4794,7 @@ deploymentSpec: \ raise ValueError(\n 'Granularity unit should be one of the\ \ following: '\n f'{valid_data_granularity_units}, got: {data_granularity_unit}.')\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-wrapped-in-list: container: args: @@ -4961,7 +4821,7 @@ deploymentSpec: - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ \ *\n\ndef wrapped_in_list(value: str) -> List[str]:\n \"\"\"Wraps a string\ \ in a list.\"\"\"\n return [value]\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 pipelineInfo: description: Trains a BQML ARIMA_PLUS model. name: automl-tabular-bqml-arima-train diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_predict_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_predict_pipeline.yaml index 1675c7c0d7..168410ffcc 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_predict_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_predict_pipeline.yaml @@ -1461,7 +1461,7 @@ deploymentSpec: \ = client.create_dataset(dataset=dataset, exists_ok=exists_ok)\n return\ \ collections.namedtuple('Outputs', ['project_id', 'dataset_id'])(\n \ \ ref.project, ref.dataset_id)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-delete-dataset-with-prefix: container: args: @@ -1495,7 +1495,7 @@ deploymentSpec: \ if dataset.dataset_id.startswith(dataset_prefix):\n client.delete_dataset(\n\ \ dataset=dataset.dataset_id,\n delete_contents=delete_contents)\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-query-job: container: args: @@ -1583,7 +1583,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-build-job-configuration-query-2: container: args: @@ -1617,7 +1617,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-first-valid: container: args: @@ -1647,7 +1647,7 @@ deploymentSpec: \ import json\n # pylint: enable=g-import-not-at-top,import-outside-toplevel,redefined-outer-name,reimported\n\ \n for value in json.loads(values):\n if value:\n return value\n\ \ raise ValueError('No valid values.')\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-table-location: container: args: @@ -1683,7 +1683,7 @@ deploymentSpec: \ if table.startswith('bq://'):\n table = table[len('bq://'):]\n elif\ \ table.startswith('bigquery://'):\n table = table[len('bigquery://'):]\n\ \ return client.get_table(table).location\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-table-location-2: container: args: @@ -1719,7 +1719,7 @@ deploymentSpec: \ if table.startswith('bq://'):\n table = table[len('bq://'):]\n elif\ \ table.startswith('bigquery://'):\n table = table[len('bigquery://'):]\n\ \ return client.get_table(table).location\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-load-table-from-uri: container: args: @@ -1760,7 +1760,7 @@ deploymentSpec: \ source_format=source_format)\n client.load_table_from_uri(\n source_uris=csv_list,\n\ \ destination=destination,\n project=project,\n location=location,\n\ \ job_config=job_config).result()\n return destination\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-make-vertex-model-artifact: container: args: @@ -1790,7 +1790,7 @@ deploymentSpec: Creates a google.VertexModel artifact.\"\"\"\n vertex_model.metadata =\ \ {'resourceName': model_resource_name}\n vertex_model.uri = (f'https://{location}-aiplatform.googleapis.com'\n\ \ f'/v1/{model_resource_name}')\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-maybe-replace-with-default: container: args: @@ -1818,7 +1818,7 @@ deploymentSpec: \ *\n\ndef maybe_replace_with_default(value: str, default: str = '') ->\ \ str:\n \"\"\"Replaces string with another value if it is a dash.\"\"\"\ \n return default if not value else value\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-model-batch-predict: container: args: @@ -1903,7 +1903,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-table-to-uri-2: container: args: @@ -1939,7 +1939,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-validate-inputs: container: args: @@ -2041,7 +2041,7 @@ deploymentSpec: \ raise ValueError(\n 'Granularity unit should be one of the\ \ following: '\n f'{valid_data_granularity_units}, got: {data_granularity_unit}.')\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 pipelineInfo: description: Creates a batch prediction using a Prophet model. name: prophet-predict diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer.py index ee9ed4ef6d..7286bf9d62 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer.py @@ -108,17 +108,17 @@ def prophet_trainer( '"machine_spec": {"machine_type": "n1-standard-4"}, ', ( '"container_spec":' - ' {"image_uri":"us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", ' + ' {"image_uri":"us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", ' ), '"args": ["prophet_trainer", "', ( f'--job_name=dataflow-{dsl.PIPELINE_JOB_NAME_PLACEHOLDER}", "' ), ( - '--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", "' + '--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "' ), ( - '--prediction_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/fte-prediction-server:20231029_0125", "' + '--prediction_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/fte-prediction-server:20240119_0125", "' ), '--artifacts_dir=', root_dir, diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer_pipeline.yaml index 3bd76df0d3..6ada0c81fe 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/forecasting/prophet_trainer_pipeline.yaml @@ -783,159 +783,125 @@ components: parameterType: BOOLEAN bigquery_staging_full_dataset_id: defaultValue: '' - description: 'Dataset in - - "projectId.datasetId" format for storing intermediate-FTE BigQuery - - tables. If the specified dataset does not exist in BigQuery, FTE will - - create the dataset. If no bigquery_staging_full_dataset_id is specified, - - all intermediate tables will be stored in a dataset created under the - - provided project in the input data source''s location during FTE - - execution called - - "vertex_feature_transform_engine_staging_{location.replace(''-'', ''_'')}". - - All tables generated by FTE will have a 30 day TTL.' + description: Dataset in "projectId.datasetId" format for storing intermediate-FTE + BigQuery tables. If the specified dataset does not exist in BigQuery, + FTE will create the dataset. If no bigquery_staging_full_dataset_id is + specified, all intermediate tables will be stored in a dataset created + under the provided project in the input data source's location during + FTE execution called "vertex_feature_transform_engine_staging_{location.replace('-', + '_')}". All tables generated by FTE will have a 30 day TTL. isOptional: true parameterType: STRING data_source_bigquery_table_path: defaultValue: '' - description: 'BigQuery input data - - source to run feature transform on.' + description: BigQuery input data source to run feature transform on. isOptional: true parameterType: STRING data_source_csv_filenames: defaultValue: '' - description: 'CSV input data source to run - - feature transform on.' + description: CSV input data source to run feature transform on. isOptional: true parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - Dataflow jobs.' + description: Custom service account to run Dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN dataset_level_custom_transformation_definitions: defaultValue: [] - description: "List of dataset-level custom transformation definitions. \ - \ Custom,\nbring-your-own dataset-level transform functions, where users\ - \ can define\nand import their own transform function and use it with\ - \ FTE's built-in\ntransformations. Using custom transformations is an\ - \ experimental feature\nand it is currently not supported during batch\ - \ prediction.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"ConcatCols\",\n \"module_path\": \"/path/to/custom_transform_fn_dlt.py\"\ - ,\n \"function_name\": \"concat_cols\" } ] Using custom transform\ - \ function\n together with FTE's built-in transformations: .. code-block::\n\ - \ python [ { \"transformation\": \"Join\", \"right_table_uri\":\n\ - \ \"bq://test-project.dataset_test.table\", \"join_keys\":\n [[\"\ - join_key_col\", \"join_key_col\"]] },{ \"transformation\":\n \"ConcatCols\"\ - , \"cols\": [\"feature_1\", \"feature_2\"], \"output_col\":\n \"feature_1_2\"\ - \ } ]" + description: 'List of dataset-level custom transformation definitions. Custom, + bring-your-own dataset-level transform functions, where users can define + and import their own transform function and use it with FTE''s built-in + transformations. Using custom transformations is an experimental feature + and it is currently not supported during batch prediction. + + [ { "transformation": "ConcatCols", "module_path": "/path/to/custom_transform_fn_dlt.py", + "function_name": "concat_cols" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "Join", "right_table_uri": "bq://test-project.dataset_test.table", "join_keys": + [["join_key_col", "join_key_col"]] },{ "transformation": "ConcatCols", + "cols": ["feature_1", "feature_2"], "output_col": "feature_1_2" } ]' isOptional: true parameterType: LIST dataset_level_transformations: defaultValue: [] - description: "List of dataset-level\ntransformations.\nExample: .. code-block::\ - \ python [ { \"transformation\": \"Join\",\n \"right_table_uri\": \"\ - bq://test-project.dataset_test.table\",\n \"join_keys\": [[\"join_key_col\"\ - , \"join_key_col\"]] }, ... ] Additional\n information about FTE's currently\ - \ supported built-in\n transformations:\n Join: Joins features from\ - \ right_table_uri. For each join key, the\n left table keys will\ - \ be included and the right table keys will\n be dropped.\n \ - \ Example: .. code-block:: python { \"transformation\": \"Join\",\n\ - \ \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ - ,\n \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }\n\ - \ Arguments:\n right_table_uri: Right table BigQuery\ - \ uri to join\n with input_full_table_id.\n join_keys:\ - \ Features to join on. For each\n nested list, the first\ - \ element is a left table column\n and the second is its\ - \ corresponding right table column.\n TimeAggregate: Creates a new\ - \ feature composed of values of an\n existing feature from a fixed\ - \ time period ago or in the future.\n Ex: A feature for sales by\ - \ store 1 year ago.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"TimeAggregate\", \"time_difference\": 40,\n \"\ - time_difference_units\": \"DAY\",\n \"time_series_identifier_columns\"\ - : [\"store_id\"],\n \"time_column\": \"time_col\", \"time_difference_target_column\"\ - :\n \"target_col\", \"output_column\": \"output_col\" }\n \ - \ Arguments:\n time_difference: Number of time_difference_units\ - \ to\n look back or into the future on our\n \ - \ time_difference_target_column.\n time_difference_units:\ - \ Units of time_difference to\n look back or into the future\ - \ on our\n time_difference_target_column. Must be one of\ - \ * 'DAY' *\n 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER'\ - \ *\n 'YEAR'\n time_series_identifier_columns:\ - \ Names of the\n time series identifier columns.\n \ - \ time_column: Name of the time column.\n time_difference_target_column:\ - \ Column we wish to get\n the value of time_difference time_difference_units\ - \ in\n the past or future.\n output_column: Name\ - \ of our new time aggregate\n feature.\n is_future:\ - \ Whether we wish to look\n forward in time. Defaults to\ - \ False.\n PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\n\ - \ Performs a partition by reduce operation (one of max,\n\ - \ min, avg, or sum) with a fixed historic time period. Ex:\n\ - \ Getting avg sales (the reduce column) for each store\n\ - \ (partition_by_column) over the previous 5 days\n \ - \ (time_column, time_ago_units, and time_ago).\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"PartitionByMax\"\ - , \"reduce_column\": \"sell_price\",\n \"partition_by_columns\"\ - : [\"store_id\", \"state_id\"],\n \"time_column\": \"date\",\ - \ \"time_ago\": 1, \"time_ago_units\":\n \"WEEK\", \"output_column\"\ - : \"partition_by_reduce_max_output\" }\n Arguments:\n \ - \ reduce_column: Column to apply the reduce operation\n \ - \ on. Reduce operations include the\n following: Max,\ - \ Min, Avg, Sum.\n partition_by_columns: List of columns to\n\ - \ partition by.\n time_column: Time column for\ - \ the partition by\n operation's window function.\n \ - \ time_ago: Number of time_ago_units to look back on\n \ - \ our target_column, starting from time_column\n (inclusive).\n\ - \ time_ago_units: Units of time_ago to look back on\n \ - \ our target_column. Must be one of * 'DAY' * 'WEEK'\n \ - \ output_column: Name of our output feature." + description: "List of dataset-level transformations.\n[ { \"transformation\"\ + : \"Join\", \"right_table_uri\": \"bq://test-project.dataset_test.table\"\ + , \"join_keys\": [[\"join_key_col\", \"join_key_col\"]] }, ... ] Additional\ + \ information about FTE's currently supported built-in\n transformations:\n\ + \ Join: Joins features from right_table_uri. For each join key, the\ + \ left table keys will be included and the right table keys will be dropped.\n\ + \ Example: .. code-block:: python { \"transformation\": \"Join\"\ + , \"right_table_uri\": \"bq://test-project.dataset_test.table\", \"join_keys\"\ + : [[\"join_key_col\", \"join_key_col\"]] }\n Arguments:\n \ + \ right_table_uri: Right table BigQuery uri to join with input_full_table_id.\n\ + \ join_keys: Features to join on. For each nested list, the\ + \ first element is a left table column and the second is its corresponding\ + \ right table column.\n TimeAggregate: Creates a new feature composed\ + \ of values of an existing feature from a fixed time period ago or in\ + \ the future.\n Ex: A feature for sales by store 1 year ago.\n \ + \ Example: .. code-block:: python { \"transformation\": \"TimeAggregate\"\ + , \"time_difference\": 40, \"time_difference_units\": \"DAY\", \"time_series_identifier_columns\"\ + : [\"store_id\"], \"time_column\": \"time_col\", \"time_difference_target_column\"\ + : \"target_col\", \"output_column\": \"output_col\" }\n Arguments:\n\ + \ time_difference: Number of time_difference_units to look\ + \ back or into the future on our time_difference_target_column.\n \ + \ time_difference_units: Units of time_difference to look back\ + \ or into the future on our time_difference_target_column. Must be one\ + \ of * 'DAY' * 'WEEK' (Equivalent to 7 DAYs) * 'MONTH' * 'QUARTER' * 'YEAR'\n\ + \ time_series_identifier_columns: Names of the time series\ + \ identifier columns.\n time_column: Name of the time column.\n\ + \ time_difference_target_column: Column we wish to get the\ + \ value of time_difference time_difference_units in the past or future.\n\ + \ output_column: Name of our new time aggregate feature.\n\ + \ is_future: Whether we wish to look forward in time. Defaults\ + \ to False. PartitionByMax/PartitionByMin/PartitionByAvg/PartitionBySum:\ + \ Performs a partition by reduce operation (one of max, min, avg, or sum)\ + \ with a fixed historic time period. Ex: Getting avg sales (the reduce\ + \ column) for each store (partition_by_column) over the previous 5 days\ + \ (time_column, time_ago_units, and time_ago).\n Example: .. code-block::\ + \ python { \"transformation\": \"PartitionByMax\", \"reduce_column\"\ + : \"sell_price\", \"partition_by_columns\": [\"store_id\", \"state_id\"\ + ], \"time_column\": \"date\", \"time_ago\": 1, \"time_ago_units\": \"\ + WEEK\", \"output_column\": \"partition_by_reduce_max_output\" }\n \ + \ Arguments:\n reduce_column: Column to apply the reduce\ + \ operation on. Reduce operations include the\n following:\ + \ Max, Min, Avg, Sum.\n partition_by_columns: List of columns\ + \ to partition by.\n time_column: Time column for the partition\ + \ by operation's window function.\n time_ago: Number of time_ago_units\ + \ to look back on our target_column, starting from time_column (inclusive).\n\ + \ time_ago_units: Units of time_ago to look back on our target_column.\ + \ Must be one of * 'DAY' * 'WEEK'\n output_column: Name of\ + \ our output feature." isOptional: true parameterType: LIST encryption_spec_key_name: @@ -945,24 +911,22 @@ components: parameterType: STRING feature_selection_algorithm: defaultValue: AMI - description: "The algorithm of feature\nselection. One of \"AMI\", \"CMIM\"\ - , \"JMIM\", \"MRMR\", default to be \"AMI\".\nThe algorithms available\ - \ are: AMI(Adjusted Mutual Information):\n Reference:\n https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\n\ - \ Arrays are not yet supported in this algorithm. CMIM(Conditional\n\ - \ Mutual Information Maximization): Reference paper: Mohamed\n \ - \ Bennasar, Yulia Hicks, Rossitza Setchi, \u201CFeature selection\ - \ using\n Joint Mutual Information Maximisation,\u201D Expert Systems\ - \ with\n Applications, vol. 42, issue 22, 1 December 2015, Pages\n\ - \ 8520-8532. JMIM(Joint Mutual Information Maximization): Reference\n\ - \ paper: Mohamed Bennasar, Yulia Hicks, Rossitza Setchi, \u201C\ - Feature\n selection using Joint Mutual Information Maximisation,\u201D\ - \ Expert\n Systems with Applications, vol. 42, issue 22, 1 December\ - \ 2015,\n Pages 8520-8532. MRMR(MIQ Minimum-redundancy\n \ - \ Maximum-relevance): Reference paper: Hanchuan Peng, Fuhui Long,\n\ - \ and Chris Ding. \"Feature selection based on mutual information\n\ - \ criteria of max-dependency, max-relevance, and min-redundancy.\"\ - \n IEEE Transactions on pattern analysis and machine intelligence\n\ - \ 27, no.\n 8: 1226-1238." + description: "The algorithm of feature selection. One of \"AMI\", \"CMIM\"\ + , \"JMIM\", \"MRMR\", default to be \"AMI\". The algorithms available\ + \ are: AMI(Adjusted Mutual Information):\nReference: https://scikit-learn.org/stable/modules/generated/sklearn.metrics.adjusted_mutual_info_score.html\ + \ Arrays are not yet supported in this algorithm. CMIM(Conditional Mutual\ + \ Information Maximization): Reference paper: Mohamed Bennasar, Yulia\ + \ Hicks, Rossitza Setchi, \u201CFeature selection using Joint Mutual Information\ + \ Maximisation,\u201D Expert Systems with Applications, vol. 42, issue\ + \ 22, 1 December 2015, Pages 8520-8532. JMIM(Joint Mutual Information\ + \ Maximization\nReference:\n paper: Mohamed Bennasar, Yulia Hicks, Rossitza\ + \ Setchi, \u201CFeature selection using Joint Mutual Information Maximisation,\u201D\ + \ Expert Systems with Applications, vol. 42, issue 22, 1 December 2015,\ + \ Pages 8520-8532. MRMR(MIQ Minimum-redundancy Maximum-relevance): Reference\ + \ paper: Hanchuan Peng, Fuhui Long, and Chris Ding. \"Feature selection\ + \ based on mutual information criteria of max-dependency, max-relevance,\ + \ and min-redundancy.\" IEEE Transactions on pattern analysis and machine\ + \ intelligence 27, no.\n 8: 1226-1238." isOptional: true parameterType: STRING feature_selection_execution_engine: @@ -978,9 +942,7 @@ components: parameterType: BOOLEAN forecasting_available_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - available at forecast columns.' + description: Forecasting available at forecast columns. isOptional: true parameterType: LIST forecasting_context_window: @@ -995,17 +957,11 @@ components: parameterType: NUMBER_INTEGER forecasting_holiday_regions: defaultValue: [] - description: 'The geographical region based on which the - - holiday effect is applied in modeling by adding holiday categorical - - array feature that include all holidays matching the date. This option - - only allowed when data granularity is day. By default, holiday effect - - modeling is disabled. To turn it on, specify the holiday region using - - this option. + description: 'The geographical region based on which the holiday effect + is applied in modeling by adding holiday categorical array feature that + include all holidays matching the date. This option only allowed when + data granularity is day. By default, holiday effect modeling is disabled. + To turn it on, specify the holiday region using this option. Top level: * ''GLOBAL'' @@ -1055,18 +1011,13 @@ components: parameterType: STRING forecasting_time_series_attribute_columns: defaultValue: [] - description: 'Forecasting - - time series attribute columns.' + description: Forecasting time series attribute columns. isOptional: true parameterType: LIST forecasting_time_series_identifier_column: description: '[Deprecated] A forecasting time series identifier column. - Raises an - - exception if used - use the "time_series_identifier_column" field - - instead.' + Raises an exception if used - use the "time_series_identifier_column" + field instead.' isOptional: true parameterType: STRING forecasting_time_series_identifier_columns: @@ -1076,9 +1027,7 @@ components: parameterType: LIST forecasting_unavailable_at_forecast_columns: defaultValue: [] - description: 'Forecasting - - unavailable at forecast columns.' + description: Forecasting unavailable at forecast columns. isOptional: true parameterType: LIST forecasting_window_max_count: @@ -1111,67 +1060,46 @@ components: parameterType: STRING materialized_examples_format: defaultValue: tfrecords_gzip - description: 'The format to use for the - - materialized examples. Should be either ''tfrecords_gzip'' (default) or - - ''parquet''.' + description: The format to use for the materialized examples. Should be + either 'tfrecords_gzip' (default) or 'parquet'. isOptional: true parameterType: STRING max_selected_features: defaultValue: 1000.0 - description: 'Maximum number of features to - - select. If specified, the transform config will be purged by only using - - the selected features that ranked top in the feature ranking, which has - - the ranking value for all supported features. If the number of input - - features is smaller than max_selected_features specified, we will still - - run the feature selection process and generate the feature ranking, no - - features will be excluded. The value will be set to 1000 by default if - - run_feature_selection is enabled.' + description: Maximum number of features to select. If specified, the transform + config will be purged by only using the selected features that ranked + top in the feature ranking, which has the ranking value for all supported + features. If the number of input features is smaller than max_selected_features + specified, we will still run the feature selection process and generate + the feature ranking, no features will be excluded. The value will be + set to 1000 by default if run_feature_selection is enabled. isOptional: true parameterType: NUMBER_INTEGER model_type: - description: 'Model type, which we wish to engineer features - - for. Can be one of: neural_network, boosted_trees, l2l, seq2seq, tft, - or - - tide. Defaults to the empty value, `None`.' + description: 'Model type, which we wish to engineer features for. Can be + one of: neural_network, boosted_trees, l2l, seq2seq, tft, or tide. Defaults + to the empty value, `None`.' isOptional: true parameterType: STRING multimodal_image_columns: defaultValue: [] - description: 'List of multimodal image - - columns. Defaults to an empty list.' + description: List of multimodal image columns. Defaults to an empty list. isOptional: true parameterType: LIST multimodal_tabular_columns: defaultValue: [] - description: 'List of multimodal tabular - - columns. Defaults to an empty list' + description: List of multimodal tabular columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_text_columns: defaultValue: [] - description: 'List of multimodal text - - columns. Defaults to an empty list' + description: List of multimodal text columns. Defaults to an empty list isOptional: true parameterType: LIST multimodal_timeseries_columns: defaultValue: [] - description: 'List of multimodal timeseries - - columns. Defaults to an empty list' + description: List of multimodal timeseries columns. Defaults to an empty + list isOptional: true parameterType: LIST predefined_split_key: @@ -1181,9 +1109,8 @@ components: parameterType: STRING prediction_type: defaultValue: '' - description: 'Model prediction type. One of - - "classification", "regression", "time_series".' + description: Model prediction type. One of "classification", "regression", + "time_series". isOptional: true parameterType: STRING project: @@ -1194,25 +1121,20 @@ components: parameterType: STRING run_distill: defaultValue: false - description: '(deprecated) Whether the distillation should be applied - - to the training.' + description: (deprecated) Whether the distillation should be applied to + the training. isOptional: true parameterType: BOOLEAN run_feature_selection: defaultValue: false - description: 'Whether the feature selection - - should be applied to the dataset.' + description: Whether the feature selection should be applied to the dataset. isOptional: true parameterType: BOOLEAN stats_gen_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - statistics generation. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental.' + description: 'Execution engine to perform statistics generation. Can be + one of: "dataflow" (by default) or "bigquery". Using "bigquery" as the + execution engine is experimental.' isOptional: true parameterType: STRING stratified_split_key: @@ -1236,264 +1158,212 @@ components: parameterType: NUMBER_DOUBLE tf_auto_transform_features: defaultValue: {} - description: "Dict mapping auto and/or type-resolutions to\nTF transform\ - \ features. FTE will automatically configure a set of\nbuilt-in transformations\ - \ for each feature based on its data statistics.\nIf users do not want\ - \ auto type resolution, but want the set of\ntransformations for a given\ - \ type to be automatically generated, they\nmay specify pre-resolved transformations\ - \ types. The following type hint\ndict keys are supported: * 'auto' *\ - \ 'categorical' * 'numeric' * 'text'\n* 'timestamp'\n Example: .. code-block::\ - \ python { \"auto\": [\"feature1\"],\n \"categorical\": [\"feature2\"\ - , \"feature3\"], } Note that the target and\n weight column may not\ - \ be included as an auto transformation unless\n users are running\ - \ forecasting." + description: 'Dict mapping auto and/or type-resolutions to TF transform + features. FTE will automatically configure a set of built-in transformations + for each feature based on its data statistics. If users do not want auto + type resolution, but want the set of transformations for a given type + to be automatically generated, they may specify pre-resolved transformations + types. The following type hint dict keys are supported: * ''auto'' * ''categorical'' + * ''numeric'' * ''text'' * ''timestamp'' Example: `{ "auto": ["feature1"], + "categorical": ["feature2", "feature3"], }`. Note that the target and + weight column may not be included as an auto transformation unless users + are running forecasting.' isOptional: true parameterType: STRUCT tf_custom_transformation_definitions: defaultValue: [] - description: "List of\nTensorFlow-based custom transformation definitions.\ - \ Custom,\nbring-your-own transform functions, where users can define\ - \ and import\ntheir own transform function and use it with FTE's built-in\n\ - transformations.\n Example: .. code-block:: python [ { \"transformation\"\ - : \"PlusOne\",\n \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"plus_one_transform\" }, { \"transformation\"\ - :\n \"MultiplyTwo\", \"module_path\": \"gs://bucket/custom_transform_fn.py\"\ - ,\n \"function_name\": \"multiply_two_transform\" } ] Using custom\n\ - \ transform function together with FTE's built-in transformations:\ - \ ..\n code-block:: python [ { \"transformation\": \"CastToFloat\"\ - ,\n \"input_columns\": [\"feature_1\"], \"output_columns\": [\"feature_1\"\ - ] },{\n \"transformation\": \"PlusOne\", \"input_columns\": [\"feature_1\"\ - ]\n \"output_columns\": [\"feature_1_plused_one\"] },{ \"transformation\"\ - :\n \"MultiplyTwo\", \"input_columns\": [\"feature_1\"] \"output_columns\"\ - :\n [\"feature_1_multiplied_two\"] } ]" + description: 'List of TensorFlow-based custom transformation definitions. Custom, + bring-your-own transform functions, where users can define and import + their own transform function and use it with FTE''s built-in transformations. + `[ { "transformation": "PlusOne", "module_path": "gs://bucket/custom_transform_fn.py", + "function_name": "plus_one_transform" }, { "transformation": "MultiplyTwo", + "module_path": "gs://bucket/custom_transform_fn.py", "function_name": + "multiply_two_transform" } ] Using custom transform function together + with FTE''s built-in transformations: .. code-block:: python [ { "transformation": + "CastToFloat", "input_columns": ["feature_1"], "output_columns": ["feature_1"] + },{ "transformation": "PlusOne", "input_columns": ["feature_1"] "output_columns": + ["feature_1_plused_one"] },{ "transformation": "MultiplyTwo", "input_columns": + ["feature_1"] "output_columns": ["feature_1_multiplied_two"] } ]' isOptional: true parameterType: LIST tf_transform_execution_engine: defaultValue: dataflow - description: 'Execution engine to perform - - row-level TF transformations. Can be one of: "dataflow" (by default) or - - "bigquery". Using "bigquery" as the execution engine is experimental and - - is for allowlisted customers only. In addition, executing on "bigquery" - - only supports auto transformations (i.e., specified by - - tf_auto_transform_features) and will raise an error when - - tf_custom_transformation_definitions or tf_transformations_path is set.' + description: 'Execution engine to perform row-level TF transformations. + Can be one of: "dataflow" (by default) or "bigquery". Using "bigquery" + as the execution engine is experimental and is for allowlisted customers + only. In addition, executing on "bigquery" only supports auto transformations + (i.e., specified by tf_auto_transform_features) and will raise an error + when tf_custom_transformation_definitions or tf_transformations_path is + set.' isOptional: true parameterType: STRING tf_transformations_path: defaultValue: '' - description: "Path to TensorFlow-based\ntransformation configuration. Path\ - \ to a JSON file used to specified\nFTE's TF transformation configurations.\ - \ In the following, we provide\nsome sample transform configurations\ - \ to demonstrate FTE's capabilities.\nAll transformations on input columns\ - \ are explicitly specified with FTE's\nbuilt-in transformations. Chaining\ - \ of multiple transformations on a\nsingle column is also supported. For\ - \ example: .. code-block:: python [\n{ \"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }, {\n\"transformation\": \"ZScale\"\ - , \"input_columns\": [\"feature_2\"] } ]\nAdditional information about\ - \ FTE's currently supported built-in\ntransformations:\n Datetime:\ - \ Extracts datetime featues from a column containing\n timestamp\ - \ strings.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"Datetime\", \"input_columns\": [\"feature_1\"], \"time_format\"\ - :\n \"%Y-%m-%d\" }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the datetime\ - \ transformation on.\n output_columns: Names of output\n\ - \ columns, one for each datetime_features element.\n \ - \ time_format: Datetime format string. Time format is\n \ - \ a combination of Date + Time Delimiter (optional) + Time\n\ - \ (optional) directives. Valid date directives are as\n\ - \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' #\n\ - \ 2018/11/30 * '%y-%m-%d' # 18-11-30 * '%y/%m/%d' #\n\ - \ 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y' #\n\ - \ 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' #\n\ - \ 11/30/18 * '%d-%m-%Y' # 30-11-2018 * '%d/%m/%Y' #\n\ - \ 30/11/2018 * '%d-%B-%Y' # 30-November-2018 * '%d-%m-%y'\n\ - \ # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' #\n\ - \ 30-November-18 * '%d%m%Y' # 30112018 * '%m%d%Y' \ - \ #\n 11302018 * '%Y%m%d' # 20181130 Valid time delimiters\n\ - \ are as follows * 'T' * ' ' Valid time directives are\ - \ as\n follows * '%H:%M' # 23:59 * '%H:%M:%S'\ - \ #\n 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456]\ - \ *\n '%H:%M:%S.%f%z' # 23:59:58[.123456]+0000 *\n \ - \ '%H:%M:%S%z', # 23:59:58+0000\n datetime_features:\ - \ List of datetime\n features to be extract. Each entry\ - \ must be one of *\n 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK'\ - \ * 'DAY_OF_YEAR'\n * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR'\ - \ * 'MINUTE' *\n 'SECOND' Defaults to ['YEAR', 'MONTH',\ - \ 'DAY',\n 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ - \ Log: Performs the natural log on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Log\",\n \ - \ \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the log transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n ZScale:\ - \ Performs Z-scale normalization on a numeric column.\n Example:\ - \ .. code-block:: python { \"transformation\":\n \"ZScale\"\ - , \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform the z-scale transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n Vocabulary:\ - \ Converts strings to integers, where each unique string\n gets\ - \ a unique integer representation.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"Vocabulary\", \"input_columns\"\ - : [\"feature_1\"] }\n Arguments:\n input_columns:\ - \ A list with a single column to\n perform the vocabulary\ - \ transformation on.\n output_columns: A list with a single\n\ - \ output column name, corresponding to the output of our\n\ - \ transformation.\n top_k: Number of the most\ - \ frequent words\n in the vocabulary to use for generating\ - \ dictionary\n lookup indices. If not specified, all words\ - \ in the\n vocabulary will be used. Defaults to None.\n\ - \ frequency_threshold: Limit the vocabulary\n \ - \ only to words whose number of occurrences in the input\n \ - \ exceeds frequency_threshold. If not specified, all words\n \ - \ in the vocabulary will be included. If both top_k and\n\ - \ frequency_threshold are specified, a word must satisfy\n\ - \ both conditions to be included. Defaults to None.\n \ - \ Categorical: Transforms categorical columns to integer columns.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Categorical\", \"input_columns\": [\"feature_1\"], \"top_k\"\ - : 10 }\n Arguments:\n input_columns: A list with\ - \ a single column to\n perform the categorical transformation\ - \ on.\n output_columns: A list with a single\n \ - \ output column name, corresponding to the output of our\n \ - \ transformation.\n top_k: Number of the most frequent\ - \ words\n in the vocabulary to use for generating dictionary\n\ - \ lookup indices. If not specified, all words in the\n\ - \ vocabulary will be used.\n frequency_threshold:\ - \ Limit the vocabulary\n only to words whose number of\ - \ occurrences in the input\n exceeds frequency_threshold.\ - \ If not specified, all words\n in the vocabulary will\ - \ be included. If both top_k and\n frequency_threshold\ - \ are specified, a word must satisfy\n both conditions\ - \ to be included.\n Reduce: Given a column where each entry is a\ - \ numeric array,\n reduces arrays according to our reduce_mode.\n\ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"Reduce\", \"input_columns\": [\"feature_1\"], \"reduce_mode\"\ - :\n \"MEAN\", \"output_columns\": [\"feature_1_mean\"] }\n\ - \ Arguments:\n input_columns: A list with a single\ - \ column to\n perform the reduce transformation on.\n \ - \ output_columns: A list with a single\n output\ - \ column name, corresponding to the output of our\n transformation.\n\ - \ reduce_mode: One of * 'MAX' * 'MIN' *\n \ - \ 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k: The number\ - \ of last k elements when\n 'LAST_K' reduce mode is used.\ - \ Defaults to 1.\n SplitString: Given a column of strings, splits\ - \ strings into token\n arrays.\n Example: .. code-block::\ - \ python { \"transformation\":\n \"SplitString\", \"input_columns\"\ - : [\"feature_1\"], \"separator\":\n \"$\" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the split string transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ separator: Separator to split input string\n into tokens.\ - \ Defaults to ' '.\n missing_token: Missing token to use\ - \ when\n no string is included. Defaults to ' _MISSING_\ - \ '.\n NGram: Given a column of strings, splits strings into token\ - \ arrays\n where each token is an integer.\n Example:\ - \ .. code-block:: python { \"transformation\": \"NGram\",\n \ - \ \"input_columns\": [\"feature_1\"], \"min_ngram_size\": 1,\n \ - \ \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_ngram_size: Minimum n-gram size. Must\n be a positive\ - \ number and <= max_ngram_size. Defaults to\n 1.\n \ - \ max_ngram_size: Maximum n-gram size. Must\n \ - \ be a positive number and >= min_ngram_size. Defaults to\n \ - \ 2.\n top_k: Number of the most frequent words\n \ - \ in the vocabulary to use for generating dictionary\n \ - \ lookup indices. If not specified, all words in the\n \ - \ vocabulary will be used. Defaults to None.\n \ - \ frequency_threshold: Limit the\n dictionary's vocabulary\ - \ only to words whose number of\n occurrences in the input\ - \ exceeds frequency_threshold. If\n not specified, all\ - \ words in the vocabulary will be\n included. If both top_k\ - \ and frequency_threshold are\n specified, a word must\ - \ satisfy both conditions to be\n included. Defaults to\ - \ None.\n separator: Separator to split input string\n \ - \ into tokens. Defaults to ' '.\n missing_token:\ - \ Missing token to use when\n no string is included. Defaults\ - \ to ' _MISSING_ '.\n Clip: Given a numeric column, clips elements\ - \ such that elements <\n min_value are assigned min_value, and\ - \ elements > max_value are\n assigned max_value.\n Example:\ - \ .. code-block:: python { \"transformation\": \"Clip\",\n \ - \ \"input_columns\": [\"col1\"], \"output_columns\":\n [\"\ - col1_clipped\"], \"min_value\": 1., \"max_value\": 10., }\n Arguments:\n\ - \ input_columns: A list with a single column to\n \ - \ perform the n-gram transformation on.\n output_columns:\ - \ A list with a single\n output column name, corresponding\ - \ to the output of our\n transformation.\n \ - \ min_value: Number where all values below\n min_value\ - \ are set to min_value. If no min_value is\n provided,\ - \ min clipping will not occur. Defaults to None.\n max_value:\ - \ Number where all values above\n max_value are set to\ - \ max_value If no max_value is\n provided, max clipping\ - \ will not occur. Defaults to None.\n MultiHotEncoding: Performs\ - \ multi-hot encoding on a categorical\n array column.\n \ - \ Example: .. code-block:: python { \"transformation\":\n \ - \ \"MultiHotEncoding\", \"input_columns\": [\"col1\"], } The number\n\ - \ of classes is determened by the largest number included in\n\ - \ the input if it is numeric or the total number of unique\n\ - \ values of the input if it is type str. If the input is has\n\ - \ type str and an element contians separator tokens, the input\n\ - \ will be split at separator indices, and the each element\ - \ of\n the split list will be considered a seperate class.\ - \ For\n example,\n Input: .. code-block:: python\ - \ [ [\"foo bar\"], # Example\n 0 [\"foo\", \"bar\"],\ - \ # Example 1 [\"foo\"], # Example\n 2 [\"bar\"\ - ], # Example 3 ]\n Output (with default separator=\"\ - \ \"): .. code-block:: python [\n [1, 1], # Example\ - \ 0 [1, 1], # Example 1\n [1, 0], # Example\ - \ 2 [0, 1], # Example 3 ]\n Arguments:\n \ - \ input_columns: A list with a single column to\n perform\ - \ the multi-hot-encoding on.\n output_columns: A list with\ - \ a single\n output column name, corresponding to the output\ - \ of our\n transformation.\n top_k: Number\ - \ of the most frequent words\n in the vocabulary to use\ - \ for generating dictionary\n lookup indices. If not specified,\ - \ all words in the\n vocabulary will be used. Defaults\ - \ to None.\n frequency_threshold: Limit the\n \ - \ dictionary's vocabulary only to words whose number of\n \ - \ occurrences in the input exceeds frequency_threshold. If\n \ - \ not specified, all words in the vocabulary will be\n \ - \ included. If both top_k and frequency_threshold are\n \ - \ specified, a word must satisfy both conditions to be\n\ - \ included. Defaults to None.\n separator:\ - \ Separator to split input string\n into tokens. Defaults\ - \ to ' '.\n MaxAbsScale: Performs maximum absolute scaling on a numeric\n\ - \ column.\n Example: .. code-block:: python { \"transformation\"\ - :\n \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\"\ - :\n [\"col1_max_abs_scaled\"] }\n Arguments:\n \ - \ input_columns: A list with a single column to\n \ - \ perform max-abs-scale on.\n output_columns: A list\ - \ with a single\n output column name, corresponding to\ - \ the output of our\n transformation.\n Custom: Transformations\ - \ defined in\n tf_custom_transformation_definitions are included\ - \ here in the\n TensorFlow-based transformation configuration.\ - \ For example,\n given the following tf_custom_transformation_definitions:\ - \ ..\n code-block:: python [ { \"transformation\": \"PlusX\"\ - ,\n \"module_path\": \"gs://bucket/custom_transform_fn.py\",\n\ - \ \"function_name\": \"plus_one_transform\" } ] We can include\ - \ the\n following transformation: .. code-block:: python {\n\ - \ \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"],\n\ - \ \"output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note\ - \ that\n input_columns must still be included in our arguments\ - \ and\n output_columns is optional. All other arguments are those\n\ - \ defined in custom_transform_fn.py, which includes `\"x\"` in\ - \ this\n case. See tf_custom_transformation_definitions above.\n\ - \ legacy_transformations_path (Optional[str]) Deprecated. Prefer\n\ - \ tf_auto_transform_features. Path to a GCS file containing JSON\n\ - \ string for legacy style transformations. Note that\n legacy_transformations_path\ - \ and tf_auto_transform_features\n cannot both be specified." + description: "Path to TensorFlow-based transformation configuration. Path\ + \ to a JSON file used to specified FTE's TF transformation configurations.\ + \ In the following, we provide some sample transform configurations to\ + \ demonstrate FTE's capabilities. All transformations on input columns\ + \ are explicitly specified with FTE's built-in transformations. Chaining\ + \ of multiple transformations on a single column is also supported. For\ + \ example: .. code-block:: python [ { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_1\"] }, { \"transformation\": \"ZScale\"\ + , \"input_columns\": [\"feature_2\"] } ]`. Additional information about\ + \ FTE's currently supported built-in\ntransformations:\nDatetime: Extracts\ + \ datetime featues from a column containing timestamp strings.\n Example:\ + \ .. code-block:: python { \"transformation\": \"Datetime\", \"input_columns\"\ + : [\"feature_1\"], \"time_format\": \"%Y-%m-%d\" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the datetime\ + \ transformation on.\n output_columns: Names of output columns,\ + \ one for each datetime_features element.\n time_format: Datetime\ + \ format string. Time format is a combination of Date + Time Delimiter\ + \ (optional) + Time (optional) directives. Valid date directives are as\ + \ follows * '%Y-%m-%d' # 2018-11-30 * '%Y/%m/%d' # 2018/11/30 * '%y-%m-%d'\ + \ # 18-11-30 * '%y/%m/%d' # 18/11/30 * '%m-%d-%Y' # 11-30-2018 * '%m/%d/%Y'\ + \ # 11/30/2018 * '%m-%d-%y' # 11-30-18 * '%m/%d/%y' # 11/30/18 * '%d-%m-%Y'\ + \ # 30-11-2018 * '%d/%m/%Y' # 30/11/2018 * '%d-%B-%Y' # 30-November-2018\ + \ * '%d-%m-%y' # 30-11-18 * '%d/%m/%y' # 30/11/18 * '%d-%B-%y' # 30-November-18\ + \ * '%d%m%Y' # 30112018 * '%m%d%Y' # 11302018 * '%Y%m%d' # 20181130\ + \ Valid time delimiters are as follows * 'T' * ' ' Valid time directives\ + \ are as follows * '%H:%M' # 23:59 * '%H:%M:%S' #\n \ + \ 23:59:58 * '%H:%M:%S.%f' # 23:59:58[.123456] * '%H:%M:%S.%f%z'\ + \ # 23:59:58[.123456]+0000 * '%H:%M:%S%z', # 23:59:58+0000\n \ + \ datetime_features: List of datetime features to be extract. Each entry\ + \ must be one of * 'YEAR' * 'MONTH' * 'DAY' * 'DAY_OF_WEEK' * 'DAY_OF_YEAR'\ + \ * 'WEEK_OF_YEAR' * 'QUARTER' * 'HOUR' * 'MINUTE' * 'SECOND' Defaults\ + \ to ['YEAR', 'MONTH', 'DAY', 'DAY_OF_WEEK', 'DAY_OF_YEAR', 'WEEK_OF_YEAR']\n\ + Log: Performs the natural log on a numeric column.\n Example: .. code-block::\ + \ python { \"transformation\": \"Log\", \"input_columns\": [\"feature_1\"\ + ] }\n Arguments:\n input_columns: A list with a single column\ + \ to perform the log transformation on.\n output_columns: A list\ + \ with a single output column name, corresponding to the output of our\ + \ transformation.\nZScale: Performs Z-scale normalization on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"ZScale\", \"input_columns\": [\"feature_1\"] }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the z-scale\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\nVocabulary:\ + \ Converts strings to integers, where each unique string gets a unique\ + \ integer representation.\n Example: .. code-block:: python { \"\ + transformation\": \"Vocabulary\", \"input_columns\": [\"feature_1\"] }\n\ + \ Arguments:\n input_columns: A list with a single column to\ + \ perform the vocabulary transformation on.\n output_columns: A\ + \ list with a single output column name, corresponding to the output of\ + \ our transformation.\n top_k: Number of the most frequent words\ + \ in the vocabulary to use for generating dictionary lookup indices. If\ + \ not specified, all words in the vocabulary will be used. Defaults to\ + \ None.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included. Defaults to None.\nCategorical: Transforms\ + \ categorical columns to integer columns.\n Example: .. code-block::\ + \ python { \"transformation\": \"Categorical\", \"input_columns\": [\"\ + feature_1\"], \"top_k\": 10 }\n Arguments:\n input_columns:\ + \ A list with a single column to perform the categorical transformation\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used.\n frequency_threshold: Limit the vocabulary only to words\ + \ whose number of occurrences in the input exceeds frequency_threshold.\ + \ If not specified, all words in the vocabulary will be included. If both\ + \ top_k and frequency_threshold are specified, a word must satisfy both\ + \ conditions to be included.\nReduce: Given a column where each entry\ + \ is a numeric array, reduces arrays according to our reduce_mode.\n \ + \ Example: .. code-block:: python { \"transformation\": \"Reduce\"\ + , \"input_columns\": [\"feature_1\"], \"reduce_mode\": \"MEAN\", \"output_columns\"\ + : [\"feature_1_mean\"] }\n Arguments:\n input_columns: A list\ + \ with a single column to perform the reduce transformation on.\n \ + \ output_columns: A list with a single output column name, corresponding\ + \ to the output of our transformation.\n reduce_mode: One of *\ + \ 'MAX' * 'MIN' * 'MEAN' * 'LAST_K' Defaults to 'MEAN'.\n last_k:\ + \ The number of last k elements when 'LAST_K' reduce mode is used. Defaults\ + \ to 1.\nSplitString: Given a column of strings, splits strings into token\ + \ arrays.\n Example: .. code-block:: python { \"transformation\"\ + : \"SplitString\", \"input_columns\": [\"feature_1\"], \"separator\":\ + \ \"$\" }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the split string transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n separator: Separator to split input\ + \ string into tokens. Defaults to ' '.\n missing_token: Missing\ + \ token to use when no string is included. Defaults to ' _MISSING_ '.\n\ + NGram: Given a column of strings, splits strings into token arrays where\ + \ each token is an integer.\n Example: .. code-block:: python { \"\ + transformation\": \"NGram\", \"input_columns\": [\"feature_1\"], \"min_ngram_size\"\ + : 1, \"max_ngram_size\": 2, \"separator\": \" \" }\n Arguments:\n \ + \ input_columns: A list with a single column to perform the n-gram\ + \ transformation on.\n output_columns: A list with a single output\ + \ column name, corresponding to the output of our transformation.\n \ + \ min_ngram_size: Minimum n-gram size. Must be a positive number\ + \ and <= max_ngram_size. Defaults to 1.\n max_ngram_size: Maximum\ + \ n-gram size. Must be a positive number and >= min_ngram_size. Defaults\ + \ to 2.\n top_k: Number of the most frequent words in the vocabulary\ + \ to use for generating dictionary lookup indices. If not specified, all\ + \ words in the vocabulary will be used. Defaults to None.\n frequency_threshold:\ + \ Limit the dictionary's vocabulary only to words whose number of occurrences\ + \ in the input exceeds frequency_threshold. If not specified, all words\ + \ in the vocabulary will be included. If both top_k and frequency_threshold\ + \ are specified, a word must satisfy both conditions to be included. Defaults\ + \ to None.\n separator: Separator to split input string into tokens.\ + \ Defaults to ' '.\n missing_token: Missing token to use when no\ + \ string is included. Defaults to ' _MISSING_ '.\nClip: Given a numeric\ + \ column, clips elements such that elements < min_value are assigned min_value,\ + \ and elements > max_value are assigned max_value.\n Example: .. code-block::\ + \ python { \"transformation\": \"Clip\", \"input_columns\": [\"col1\"\ + ], \"output_columns\": [\"col1_clipped\"], \"min_value\": 1., \"max_value\"\ + : 10., }\n Arguments:\n input_columns: A list with a single\ + \ column to perform the n-gram transformation on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\n min_value: Number where all values below\ + \ min_value are set to min_value. If no min_value is provided, min clipping\ + \ will not occur. Defaults to None.\n max_value: Number where all\ + \ values above max_value are set to max_value If no max_value is provided,\ + \ max clipping will not occur. Defaults to None.\nMultiHotEncoding: Performs\ + \ multi-hot encoding on a categorical array column.\n Example: ..\ + \ code-block:: python { \"transformation\": \"MultiHotEncoding\", \"\ + input_columns\": [\"col1\"], } The number of classes is determened by\ + \ the largest number included in the input if it is numeric or the total\ + \ number of unique values of the input if it is type str. If the input\ + \ is has type str and an element contians separator tokens, the input\ + \ will be split at separator indices, and the each element of the split\ + \ list will be considered a seperate class. For example,\n Input: \ + \ .. code-block:: python [ [\"foo bar\"], # Example 0 [\"foo\",\ + \ \"bar\"], # Example 1 [\"foo\"], # Example 2 [\"bar\"], \ + \ # Example 3 ] Output (with default separator=\" \"): .. code-block::\ + \ python [ [1, 1], # Example 0 [1, 1], # Example 1 [1,\ + \ 0], # Example 2 [0, 1], # Example 3 ]\n Arguments:\n\ + \ input_columns: A list with a single column to perform the multi-hot-encoding\ + \ on.\n output_columns: A list with a single output column name,\ + \ corresponding to the output of our transformation.\n top_k: Number\ + \ of the most frequent words in the vocabulary to use for generating dictionary\ + \ lookup indices. If not specified, all words in the vocabulary will be\ + \ used. Defaults to None.\n frequency_threshold: Limit the dictionary's\ + \ vocabulary only to words whose number of occurrences in the input exceeds\ + \ frequency_threshold. If not specified, all words in the vocabulary will\ + \ be included. If both top_k and frequency_threshold are specified, a\ + \ word must satisfy both conditions to be included. Defaults to None.\n\ + \ separator: Separator to split input string into tokens. Defaults\ + \ to ' '.\nMaxAbsScale: Performs maximum absolute scaling on a numeric\ + \ column.\n Example: .. code-block:: python { \"transformation\"\ + : \"MaxAbsScale\", \"input_columns\": [\"col1\"], \"output_columns\":\ + \ [\"col1_max_abs_scaled\"] }\n Arguments:\n input_columns:\ + \ A list with a single column to perform max-abs-scale on.\n output_columns:\ + \ A list with a single output column name, corresponding to the output\ + \ of our transformation.\nCustom: Transformations defined in tf_custom_transformation_definitions\ + \ are included here in the TensorFlow-based transformation configuration.\ + \ For example, given the following tf_custom_transformation_definitions:\ + \ .. code-block:: python [ { \"transformation\": \"PlusX\", \"module_path\"\ + : \"gs://bucket/custom_transform_fn.py\", \"function_name\": \"plus_one_transform\"\ + \ } ] We can include the following transformation: .. code-block:: python\ + \ { \"transformation\": \"PlusX\", \"input_columns\": [\"col1\"], \"\ + output_columns\": [\"col1_max_abs_scaled\"] \"x\": 5 } Note that input_columns\ + \ must still be included in our arguments and output_columns is optional.\ + \ All other arguments are those defined in custom_transform_fn.py, which\ + \ includes `\"x\"` in this case. See tf_custom_transformation_definitions\ + \ above. legacy_transformations_path (Optional[str]) Deprecated. Prefer\ + \ tf_auto_transform_features. Path to a GCS file containing JSON string\ + \ for legacy style transformations. Note that legacy_transformations_path\ + \ and tf_auto_transform_features cannot both be specified." isOptional: true parameterType: STRING timestamp_split_key: @@ -1527,11 +1397,9 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The ranking of features, all features supported in the - - dataset will be included. For "AMI" algorithm, array features won''t be - - available in the ranking as arrays are not supported yet.' + description: The ranking of features, all features supported in the dataset + will be included. For "AMI" algorithm, array features won't be available + in the ranking as arrays are not supported yet. instance_schema: artifactType: schemaTitle: system.Artifact @@ -1552,36 +1420,28 @@ components: description: The transform output artifact. parameters: bigquery_downsampled_test_split_uri: - description: 'BigQuery URI for the downsampled test - - split to pass to the batch prediction component during batch explain.' + description: BigQuery URI for the downsampled test split to pass to the + batch prediction component during batch explain. parameterType: STRING bigquery_test_split_uri: - description: 'BigQuery URI for the test split to pass to the - - batch prediction component during evaluation.' + description: BigQuery URI for the test split to pass to the batch prediction + component during evaluation. parameterType: STRING bigquery_train_split_uri: - description: 'BigQuery URI for the train split to pass to the - - batch prediction component during distillation.' + description: BigQuery URI for the train split to pass to the batch prediction + component during distillation. parameterType: STRING bigquery_validation_split_uri: - description: 'BigQuery URI for the validation split to - - pass to the batch prediction component during distillation.' + description: BigQuery URI for the validation split to pass to the batch + prediction component during distillation. parameterType: STRING gcp_resources: - description: 'GCP resources created by this component. For more details, - - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING split_example_counts: - description: 'JSON string of data split example counts for train, - - validate, and test splits.' + description: JSON string of data split example counts for train, validate, + and test splits. parameterType: STRING comp-get-fte-suffix: executorLabel: exec-get-fte-suffix @@ -1934,50 +1794,37 @@ components: inputDefinitions: parameters: data_granularity_unit: - description: 'String representing the units of time for the - - time column.' + description: String representing the units of time for the time column. parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'Dataflow worker''s disk size in GB - - during training.' + description: Dataflow worker's disk size in GB during training. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-1 - description: 'The dataflow machine type used for - - training.' + description: The dataflow machine type used for training. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 10.0 - description: 'The max number of Dataflow - - workers used for training.' + description: The max number of Dataflow workers used for training. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used.' + description: Dataflow's fully qualified subnetwork name, when empty the + default subnetwork will be used. isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -1986,37 +1833,30 @@ components: isOptional: true parameterType: STRING forecast_horizon: - description: 'The number of time periods into the future for - - which forecasts will be created. Future periods start after the latest - - timestamp for each time series.' + description: The number of time periods into the future for which forecasts + will be created. Future periods start after the latest timestamp for each + time series. parameterType: NUMBER_INTEGER location: description: The GCP region for Vertex AI. parameterType: STRING max_num_trials: defaultValue: 6.0 - description: 'Maximum number of tuning trials to perform - - per time series. There are up to 100 possible combinations to explore - - for each time series. Recommended values to try are 3, 6, and 24.' + description: Maximum number of tuning trials to perform per time series. + There are up to 100 possible combinations to explore for each time series. + Recommended values to try are 3, 6, and 24. isOptional: true parameterType: NUMBER_INTEGER optimization_objective: defaultValue: rmse - description: 'Optimization objective for tuning. Supported - - metrics come from Prophet''s performance_metrics function. These are mse, - - rmse, mae, mape, mdape, smape, and coverage.' + description: Optimization objective for tuning. Supported metrics come from + Prophet's performance_metrics function. These are mse, rmse, mae, mape, + mdape, smape, and coverage. isOptional: true parameterType: STRING predefined_split_column: - description: 'The predefined_split column name. A string - - that represents a list of comma separated CSV filenames.' + description: The predefined_split column name. A string that represents + a list of comma separated CSV filenames. parameterType: STRING project: description: The GCP project that runs the pipeline components. @@ -2025,33 +1865,21 @@ components: description: The Cloud Storage location to store the output. parameterType: STRING source_bigquery_uri: - description: 'The BigQuery table path of format - - bq (str)://bq_project.bq_dataset.bq_table' + description: The BigQuery table path of format bq (str)://bq_project.bq_dataset.bq_table parameterType: STRING target_column: - description: 'Name of the column that the model is to predict - - values for.' + description: Name of the column that the model is to predict values for. parameterType: STRING time_column: - description: 'Name of the column that identifies time order in the - - time series.' + description: Name of the column that identifies time order in the time series. parameterType: STRING time_series_identifier_column: - description: 'Name of the column that identifies - - the time series.' + description: Name of the column that identifies the time series. parameterType: STRING window_column: - description: 'Name of the column that should be used to filter - - input rows. The column should contain either booleans or string - - booleans; if the value of the row is True, generate a sliding window - - from that row.' + description: Name of the column that should be used to filter input rows. The + column should contain either booleans or string booleans; if the value + of the row is True, generate a sliding window from that row. parameterType: STRING outputDefinitions: artifacts: @@ -2066,9 +1894,8 @@ components: description: The UnmanagedContainerModel artifact. parameters: gcp_resources: - description: 'Serialized gcp_resources proto tracking the custom training - - job.' + description: Serialized gcp_resources proto tracking the custom training + job. parameterType: STRING comp-table-to-uri: executorLabel: exec-table-to-uri @@ -2194,7 +2021,7 @@ deploymentSpec: \ = client.create_dataset(dataset=dataset, exists_ok=exists_ok)\n return\ \ collections.namedtuple('Outputs', ['project_id', 'dataset_id'])(\n \ \ ref.project, ref.dataset_id)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-delete-dataset-with-prefix: container: args: @@ -2228,7 +2055,7 @@ deploymentSpec: \ if dataset.dataset_id.startswith(dataset_prefix):\n client.delete_dataset(\n\ \ dataset=dataset.dataset_id,\n delete_contents=delete_contents)\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-bigquery-query-job: container: args: @@ -2289,7 +2116,7 @@ deploymentSpec: \ 'datasetId': dataset_id,\n 'tableId': table_id,\n }\n\ \ if write_disposition:\n config['write_disposition'] = write_disposition\n\ \ return config\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-feature-transform-engine: container: args: @@ -2374,8 +2201,8 @@ deploymentSpec: "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp"]}' - '{"Concat": ["--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}"]}' - '{"Concat": ["--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}"]}' - - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125 - - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + - --dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125 + - --feature_transform_engine_docker_uri=us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 - '{"Concat": ["--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}"]}' - '{"Concat": ["--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}"]}' - '{"Concat": ["--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}"]}' @@ -2392,7 +2219,7 @@ deploymentSpec: - '{"IfPresent": {"InputName": "group_temporal_total_weight", "Then": {"Concat": ["--group_temporal_total_weight=", "{{$.inputs.parameters[''group_temporal_total_weight'']}}"]}}}' - '{"Concat": ["--encryption_spec_key_name=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}"]}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125 exec-get-fte-suffix: container: args: @@ -2426,7 +2253,7 @@ deploymentSpec: \ table.table_id.startswith(fte_table):\n return table.table_id[len(fte_table)\ \ + 1:]\n raise ValueError(\n f'No FTE output tables found in {bigquery_staging_full_dataset_id}.')\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-get-table-location: container: args: @@ -2462,7 +2289,7 @@ deploymentSpec: \ if table.startswith('bq://'):\n table = table[len('bq://'):]\n elif\ \ table.startswith('bigquery://'):\n table = table[len('bigquery://'):]\n\ \ return client.get_table(table).location\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-model-evaluation-regression: container: args: @@ -2573,10 +2400,10 @@ deploymentSpec: ", "\"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, ", "\"job_spec\": {\"worker_pool_specs\": [{\"replica_count\":\"1\", ", "\"machine_spec\": {\"machine_type\": \"n1-standard-4\"}, ", "\"container_spec\": - {\"image_uri\":\"us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125\", + {\"image_uri\":\"us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125\", ", "\"args\": [\"prophet_trainer\", \"", "--job_name=dataflow-{{$.pipeline_job_name}}\", - \"", "--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125\", - \"", "--prediction_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/fte-prediction-server:20231029_0125\", + \"", "--dataflow_worker_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125\", + \"", "--prediction_container_image=us-docker.pkg.dev/vertex-ai/automl-tabular/fte-prediction-server:20240119_0125\", \"", "--artifacts_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/model/\", \"", "--evaluated_examples_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/eval/\", \"", "--region=", "{{$.inputs.parameters[''location'']}}", @@ -2640,7 +2467,7 @@ deploymentSpec: \ if use_bq_prefix:\n bq_uri = 'bq://' + bq_uri\n outputs.append(bq_uri)\n\ \ return collections.namedtuple(\n 'Outputs',\n ['project_id',\ \ 'dataset_id', 'table_id', 'uri'],\n )(*outputs)\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-validate-inputs: container: args: @@ -2742,7 +2569,7 @@ deploymentSpec: \ raise ValueError(\n 'Granularity unit should be one of the\ \ following: '\n f'{valid_data_granularity_units}, got: {data_granularity_unit}.')\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-wrapped-in-list: container: args: @@ -2769,7 +2596,7 @@ deploymentSpec: - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ \ *\n\ndef wrapped_in_list(value: str) -> List[str]:\n \"\"\"Wraps a string\ \ in a list.\"\"\"\n return [value]\n\n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 pipelineInfo: description: Trains one Prophet model per time series. name: prophet-train diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/automl_tabular_pipeline.yaml b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/automl_tabular_pipeline.yaml index e9a3d9459a..b3a4c1ee0e 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/automl_tabular_pipeline.yaml +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/automl_tabular_pipeline.yaml @@ -113,15 +113,13 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. @@ -134,13 +132,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -155,10 +148,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-cv-trainer-2: executorLabel: exec-automl-tabular-cv-trainer-2 @@ -201,15 +192,13 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. @@ -222,13 +211,8 @@ components: parameterType: NUMBER_INTEGER worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -243,10 +227,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble: executorLabel: exec-automl-tabular-ensemble @@ -261,9 +243,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -278,18 +258,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -299,11 +275,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -349,10 +322,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble-2: executorLabel: exec-automl-tabular-ensemble-2 @@ -367,9 +338,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -384,18 +353,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -405,11 +370,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -455,10 +417,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-ensemble-3: executorLabel: exec-automl-tabular-ensemble-3 @@ -473,9 +433,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The instance baseline - - used to calculate explanations.' + description: The instance baseline used to calculate explanations. metadata: artifactType: schemaTitle: system.Artifact @@ -490,18 +448,14 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'AutoML Tabular tuning - - result.' + description: AutoML Tabular tuning result. warmup_data: artifactType: schemaTitle: system.Dataset schemaVersion: 0.0.1 - description: 'The warm up data. Ensemble component will save the - - warm up data together with the model artifact, used to warm up the model - - when prediction server starts.' + description: The warm up data. Ensemble component will save the warm up + data together with the model artifact, used to warm up the model when + prediction server starts. isOptional: true parameters: encryption_spec_key_name: @@ -511,11 +465,8 @@ components: parameterType: STRING export_additional_model_without_custom_ops: defaultValue: false - description: 'True if export - - an additional model without custom TF operators to the - - `model_without_custom_ops` output.' + description: True if export an additional model without custom TF operators + to the `model_without_custom_ops` output. isOptional: true parameterType: BOOLEAN location: @@ -561,10 +512,8 @@ components: explanation_parameters: parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-finalizer: executorLabel: exec-automl-tabular-finalizer @@ -587,10 +536,8 @@ components: outputDefinitions: parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-infra-validator: executorLabel: exec-automl-tabular-infra-validator @@ -600,9 +547,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-infra-validator-2: executorLabel: exec-automl-tabular-infra-validator-2 inputDefinitions: @@ -611,9 +556,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-infra-validator-3: executorLabel: exec-automl-tabular-infra-validator-3 inputDefinitions: @@ -622,9 +565,7 @@ components: artifactType: schemaTitle: google.UnmanagedContainerModel schemaVersion: 0.0.1 - description: 'google.UnmanagedContainerModel for model - - to be validated.' + description: google.UnmanagedContainerModel for model to be validated. comp-automl-tabular-stage-1-tuner: executorLabel: exec-automl-tabular-stage-1-tuner inputDefinitions: @@ -643,9 +584,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -658,15 +597,11 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE disable_early_stopping: defaultValue: false - description: 'True if disable early stopping. Default - - value is false.' + description: True if disable early stopping. Default value is false. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -682,24 +617,21 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -707,9 +639,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN single_run_max_secs: @@ -717,11 +647,8 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "model_type","categorical_value_spec": {"values": - - ["nn"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "model_type","categorical_value_spec": + {"values": ["nn"]}}]' isOptional: true parameterType: LIST tune_feature_selection_rate: @@ -730,13 +657,8 @@ components: parameterType: BOOLEAN worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -751,10 +673,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-stage-1-tuner-2: executorLabel: exec-automl-tabular-stage-1-tuner-2 @@ -774,9 +694,7 @@ components: artifactType: schemaTitle: system.Artifact schemaVersion: 0.0.1 - description: 'The materialized train - - split.' + description: The materialized train split. metadata: artifactType: schemaTitle: system.Artifact @@ -789,15 +707,11 @@ components: description: The transform output artifact. parameters: deadline_hours: - description: 'Number of hours the cross-validation trainer - - should run.' + description: Number of hours the cross-validation trainer should run. parameterType: NUMBER_DOUBLE disable_early_stopping: defaultValue: false - description: 'True if disable early stopping. Default - - value is false.' + description: True if disable early stopping. Default value is false. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -813,24 +727,21 @@ components: parameterType: NUMBER_INTEGER num_selected_features: defaultValue: 0.0 - description: 'Number of selected features. The number of - - features to learn in the NN models.' + description: Number of selected features. The number of features to learn + in the NN models. isOptional: true parameterType: NUMBER_INTEGER num_selected_trials: - description: 'Number of selected trials. The number of weak - - learners in the final model is 5 * num_selected_trials.' + description: Number of selected trials. The number of weak learners in the + final model is 5 * num_selected_trials. parameterType: NUMBER_INTEGER project: description: Project to run Cross-validation trainer. parameterType: STRING reduce_search_space_mode: defaultValue: regular - description: 'The reduce search space mode. Possible - - values: "regular" (default), "minimal", "full".' + description: 'The reduce search space mode. Possible values: "regular" (default), + "minimal", "full".' isOptional: true parameterType: STRING root_dir: @@ -838,9 +749,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN single_run_max_secs: @@ -848,11 +757,8 @@ components: parameterType: NUMBER_INTEGER study_spec_parameters_override: defaultValue: [] - description: 'JSON study spec. E.g., - - [{"parameter_id": "model_type","categorical_value_spec": {"values": - - ["nn"]}}]' + description: 'JSON study spec. E.g., [{"parameter_id": "model_type","categorical_value_spec": + {"values": ["nn"]}}]' isOptional: true parameterType: LIST tune_feature_selection_rate: @@ -861,13 +767,8 @@ components: parameterType: BOOLEAN worker_pool_specs_override_json: defaultValue: [] - description: 'JSON worker pool specs. E.g., - - [{"machine_spec": {"machine_type": - - "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": - - "n1-standard-16"}}]' + description: 'JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": + "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}]' isOptional: true parameterType: LIST outputDefinitions: @@ -882,10 +783,8 @@ components: description: Core metrics in dictionary of component execution. parameterType: STRUCT gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-transform: executorLabel: exec-automl-tabular-transform @@ -919,46 +818,36 @@ components: parameters: dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More - - details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -1003,10 +892,8 @@ components: description: The transform output artifact. parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-automl-tabular-transform-2: executorLabel: exec-automl-tabular-transform-2 @@ -1040,46 +927,36 @@ components: parameters: dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More - - details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN encryption_spec_key_name: @@ -1124,10 +1001,8 @@ components: description: The transform output artifact. parameters: gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING comp-bool-identity: executorLabel: exec-bool-identity @@ -8305,48 +8180,36 @@ components: parameterType: STRING dataflow_disk_size_gb: defaultValue: 40.0 - description: 'The disk size, in gigabytes, to use - - on each Dataflow worker instance. If not set, default to 40.' + description: The disk size, in gigabytes, to use on each Dataflow worker + instance. If not set, default to 40. isOptional: true parameterType: NUMBER_INTEGER dataflow_machine_type: defaultValue: n1-standard-16 - description: 'The machine type used for dataflow - - jobs. If not set, default to n1-standard-16.' + description: The machine type used for dataflow jobs. If not set, default + to n1-standard-16. isOptional: true parameterType: STRING dataflow_max_num_workers: defaultValue: 25.0 - description: 'The number of workers to run the - - dataflow job. If not set, default to 25.' + description: The number of workers to run the dataflow job. If not set, + default to 25. isOptional: true parameterType: NUMBER_INTEGER dataflow_service_account: defaultValue: '' - description: 'Custom service account to run - - dataflow jobs.' + description: Custom service account to run dataflow jobs. isOptional: true parameterType: STRING dataflow_subnetwork: defaultValue: '' - description: 'Dataflow''s fully qualified subnetwork - - name, when empty the default subnetwork will be used. More - - details: - - https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' + description: 'Dataflow''s fully qualified subnetwork name, when empty the + default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications' isOptional: true parameterType: STRING dataflow_use_public_ips: defaultValue: true - description: 'Specifies whether Dataflow - - workers use public IP addresses.' + description: Specifies whether Dataflow workers use public IP addresses. isOptional: true parameterType: BOOLEAN enable_probabilistic_inference: @@ -8359,44 +8222,36 @@ components: isOptional: true parameterType: STRING location: - description: 'Location for running dataset statistics and example - - generation.' + description: Location for running dataset statistics and example generation. parameterType: STRING optimization_objective: defaultValue: '' - description: "Objective function the model is optimizing\ntowards. The training\ - \ process creates a model that maximizes/minimizes\nthe value of the objective\ - \ function over the validation set. The\nsupported optimization objectives\ - \ depend on the prediction type. If the\nfield is not set, a default objective\ - \ function is used.\n classification: \"maximize-au-roc\" (default) -\ - \ Maximize the\n area under the receiver operating characteristic (ROC)\ - \ curve.\n \"minimize-log-loss\" - Minimize log loss. \"maximize-au-prc\"\ - \ -\n Maximize the area under the precision-recall curve.\n \"maximize-precision-at-recall\"\ - \ - Maximize precision for a specified\n recall value. \"maximize-recall-at-precision\"\ - \ - Maximize recall for a\n specified precision value.\n classification\ - \ (multi-class): \"minimize-log-loss\" (default) - Minimize\n log loss.\n\ - \ regression: \"minimize-rmse\" (default) - Minimize root-mean-squared\n\ - \ error (RMSE). \"minimize-mae\" - Minimize mean-absolute error (MAE).\n\ - \ \"minimize-rmsle\" - Minimize root-mean-squared log error (RMSLE)." + description: 'Objective function the model is optimizing towards. The training + process creates a model that maximizes/minimizes the value of the objective + function over the validation set. The supported optimization objectives + depend on the prediction type. If the field is not set, a default objective + function is used. classification: "maximize-au-roc" (default) - Maximize + the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" + - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall + curve. "maximize-precision-at-recall" - Maximize precision for a specified + recall value. "maximize-recall-at-precision" - Maximize recall for a specified + precision value. classification (multi-class): "minimize-log-loss" (default) + - Minimize log loss. regression: "minimize-rmse" (default) - Minimize + root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute + error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error + (RMSLE).' isOptional: true parameterType: STRING optimization_objective_precision_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-recall-at-precision". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-recall-at-precision". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE optimization_objective_recall_value: defaultValue: -1.0 - description: 'Required when - - optimization_objective is "maximize-precision-at-recall". Must be - - between 0 and 1, inclusive.' + description: Required when optimization_objective is "maximize-precision-at-recall". + Must be between 0 and 1, inclusive. isOptional: true parameterType: NUMBER_DOUBLE predefined_split_key: @@ -8404,14 +8259,10 @@ components: isOptional: true parameterType: STRING prediction_type: - description: 'The prediction type. Supported values: - - "classification", "regression".' + description: 'The prediction type. Supported values: "classification", "regression".' parameterType: STRING project: - description: 'Project to run dataset statistics and example - - generation.' + description: Project to run dataset statistics and example generation. parameterType: STRING quantiles: defaultValue: [] @@ -8426,9 +8277,7 @@ components: parameterType: STRING run_distillation: defaultValue: false - description: 'True if in distillation mode. The default value - - is false.' + description: True if in distillation mode. The default value is false. isOptional: true parameterType: BOOLEAN stratified_split_key: @@ -8451,21 +8300,14 @@ components: isOptional: true parameterType: NUMBER_DOUBLE transformations: - description: 'Quote escaped JSON string for transformations. Each - - transformation will apply transform function to given input column. And - - the result will be used for training. When creating transformation for - - BigQuery Struct column, the column should be flattened using "." as the - - delimiter.' + description: Quote escaped JSON string for transformations. Each transformation + will apply transform function to given input column. And the result will + be used for training. When creating transformation for BigQuery Struct + column, the column should be flattened using "." as the delimiter. parameterType: STRING transformations_path: defaultValue: '' - description: 'Path to a GCS file containing JSON - - string for transformations.' + description: Path to a GCS file containing JSON string for transformations. isOptional: true parameterType: STRING validation_fraction: @@ -8519,10 +8361,8 @@ components: description: The downsampled test split JSON object. parameterType: LIST gcp_resources: - description: 'GCP resources created by this component. For more details, - see - - https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md.' + description: GCP resources created by this component. For more details, + see https://github.com/kubeflow/pipelines/blob/master/components/google-cloud/google_cloud_pipeline_components/proto/README.md. parameterType: STRING test_split_json: description: The test split JSON object. @@ -8575,9 +8415,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_cv_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -8618,9 +8458,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_cv_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--component_id={{$.pipeline_task_uuid}}\", \"--training_base_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/train\", \"--num_parallel_trial=", "{{$.inputs.parameters[''num_parallel_trials'']}}", @@ -8661,7 +8501,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -8673,7 +8513,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -8702,7 +8542,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -8714,7 +8554,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -8743,7 +8583,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-highmem-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"ensemble\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", "\", \"--model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/model\", \"--custom_model_output_path=", "{{$.inputs.parameters[''root_dir'']}}", @@ -8755,7 +8595,7 @@ deploymentSpec: "\", \"--tuning_result_input_path=", "{{$.inputs.artifacts[''tuning_result_input''].uri}}", "\", \"--instance_baseline_path=", "{{$.inputs.artifacts[''instance_baseline''].uri}}", "\", \"--warmup_data=", "{{$.inputs.artifacts[''warmup_data''].uri}}", "\", - \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125", + \"--prediction_docker_uri=", "us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125", "\", \"--model_path=", "{{$.outputs.artifacts[''model''].uri}}", "\", \"--custom_model_path=", "{{$.outputs.artifacts[''model_without_custom_ops''].uri}}", "\", \"--explanation_metadata_path=", "{{$.outputs.parameters[''explanation_metadata''].output_file}}", ",", "{{$.outputs.artifacts[''explanation_metadata_artifact''].uri}}", @@ -8784,7 +8624,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"cancel_l2l_tuner\", \"--error_file_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/error.pb\", \"--cleanup_lro_job_infos=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/lro\"]}}]}}"]}' @@ -8799,7 +8639,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -8808,7 +8648,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -8817,7 +8657,7 @@ deploymentSpec: args: - --executor_input - '{{$}}' - image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125 resources: cpuLimit: 8.0 memoryLimit: 52.0 @@ -8837,9 +8677,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_stage_1_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--feature_selection_result_path=", "{{$.inputs.artifacts[''feature_ranking''].uri}}", "\", \"--disable_early_stopping=", "{{$.inputs.parameters[''disable_early_stopping'']}}", "\", \"--tune_feature_selection_rate=", "{{$.inputs.parameters[''tune_feature_selection_rate'']}}", @@ -8884,9 +8724,9 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"l2l_stage_1_tuner\", \"--transform_output_path=", "{{$.inputs.artifacts[''transform_output''].uri}}", - "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", + "\", \"--training_docker_uri=", "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"--feature_selection_result_path=", "{{$.inputs.artifacts[''feature_ranking''].uri}}", "\", \"--disable_early_stopping=", "{{$.inputs.parameters[''disable_early_stopping'']}}", "\", \"--tune_feature_selection_rate=", "{{$.inputs.parameters[''tune_feature_selection_rate'']}}", @@ -8931,7 +8771,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"transform\", \"--is_mp=true\", \"--transform_output_artifact_path=", "{{$.outputs.artifacts[''transform_output''].uri}}", "\", \"--transform_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/transform\", @@ -8952,7 +8792,7 @@ deploymentSpec: \"--dataflow_tmp_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp\", \"--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}", "\", \"--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}", - "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", + "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "\", \"--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}", "\", \"--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}", "\", \"--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}", @@ -8983,7 +8823,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"transform\", \"--is_mp=true\", \"--transform_output_artifact_path=", "{{$.outputs.artifacts[''transform_output''].uri}}", "\", \"--transform_output_path=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/transform\", @@ -9004,7 +8844,7 @@ deploymentSpec: \"--dataflow_tmp_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp\", \"--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}", "\", \"--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}", - "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", + "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "\", \"--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}", "\", \"--dataflow_subnetwork_fully_qualified=", "{{$.inputs.parameters[''dataflow_subnetwork'']}}", "\", \"--dataflow_use_public_ips=", "{{$.inputs.parameters[''dataflow_use_public_ips'']}}", @@ -10236,7 +10076,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-2: container: args: @@ -10265,7 +10105,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-model-upload-3: container: args: @@ -10294,7 +10134,7 @@ deploymentSpec: - -u - -m - launcher - image: gcr.io/ml-pipeline/automl-tables-private:1.0.15 + image: gcr.io/ml-pipeline/automl-tables-private:1.0.17 exec-read-input-uri: container: args: @@ -10419,7 +10259,7 @@ deploymentSpec: \ 'model_display_name',\n ],\n )(\n data_source_csv_filenames,\n\ \ data_source_bigquery_table_path,\n model_display_name,\n )\n\ \n" - image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20231029_0125 + image: us-docker.pkg.dev/vertex-ai/automl-tabular/kfp-v2-base:20240119_0125 exec-string-not-empty: container: args: @@ -10466,7 +10306,7 @@ deploymentSpec: \"encryption_spec\": {\"kms_key_name\":\"", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", "\"}, \"job_spec\": {\"worker_pool_specs\": [{\"replica_count\": 1, \"machine_spec\": {\"machine_type\": \"n1-standard-8\"}, \"container_spec\": {\"image_uri\":\"", - "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125", "\", + "us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125", "\", \"args\": [\"stats_generator\",", "\"--train_spec={\\\"prediction_type\\\": \\\"", "{{$.inputs.parameters[''prediction_type'']}}", "\\\", \\\"target_column\\\": \\\"", "{{$.inputs.parameters[''target_column_name'']}}", "\\\", \\\"optimization_objective\\\": @@ -10499,7 +10339,7 @@ deploymentSpec: \"--dataflow_staging_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_staging\", \"--dataflow_tmp_dir=", "{{$.inputs.parameters[''root_dir'']}}", "/{{$.pipeline_job_uuid}}/{{$.pipeline_task_uuid}}/dataflow_tmp\", \"--dataflow_max_num_workers=", "{{$.inputs.parameters[''dataflow_max_num_workers'']}}", - "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125", + "\", \"--dataflow_worker_container_image=", "us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125", "\", \"--dataflow_machine_type=", "{{$.inputs.parameters[''dataflow_machine_type'']}}", "\", \"--dataflow_disk_size_gb=", "{{$.inputs.parameters[''dataflow_disk_size_gb'']}}", "\", \"--dataflow_kms_key=", "{{$.inputs.parameters[''encryption_spec_key_name'']}}", diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/cv_trainer.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/cv_trainer.py index 20d5811c0b..8ad4050b5a 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/cv_trainer.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/cv_trainer.py @@ -99,11 +99,11 @@ def automl_tabular_cv_trainer( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "args": ["l2l_cv_tuner", "--transform_output_path=', transform_output.uri, '", "--training_docker_uri=', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', ( f'", "--component_id={dsl.PIPELINE_TASK_ID_PLACEHOLDER}",' ' "--training_base_dir=' diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/ensemble.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/ensemble.py index b9f7bd2a57..b2d9accb9b 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/ensemble.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/ensemble.py @@ -106,7 +106,7 @@ def automl_tabular_ensemble( ' 1, "machine_spec": {"machine_type": "n1-highmem-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "args": ["ensemble", "--transform_output_path=', transform_output.uri, '", "--model_output_path=', @@ -137,7 +137,7 @@ def automl_tabular_ensemble( '", "--warmup_data=', warmup_data.uri, '", "--prediction_docker_uri=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125', '", "--model_path=', model.uri, '", "--custom_model_path=', diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/finalizer.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/finalizer.py index c04b5e3d5a..e63c9a51de 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/finalizer.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/finalizer.py @@ -72,7 +72,7 @@ def automl_tabular_finalizer( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "args": ["cancel_l2l_tuner", "--error_file_path=', root_dir, ( diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/infra_validator.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/infra_validator.py index 21e92e219f..697c6a6684 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/infra_validator.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/infra_validator.py @@ -32,7 +32,7 @@ def automl_tabular_infra_validator( # fmt: on return dsl.ContainerSpec( - image='us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20231029_0125', + image='us-docker.pkg.dev/vertex-ai/automl-tabular/prediction-server:20240119_0125', command=[], args=['--executor_input', '{{$}}'], ) diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/split_materialized_data.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/split_materialized_data.py index ab6616be47..b4aee5d4c8 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/split_materialized_data.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/split_materialized_data.py @@ -52,7 +52,7 @@ def split_materialized_data( # fmt: on return dsl.ContainerSpec( - image='us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125', + image='us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125', command=[ 'sh', '-ec', diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stage_1_tuner.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stage_1_tuner.py index 51894e4073..d1167ff59a 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stage_1_tuner.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stage_1_tuner.py @@ -57,8 +57,7 @@ def automl_tabular_stage_1_tuner( root_dir: The Cloud Storage location to store the output. study_spec_parameters_override: JSON study spec. E.g., [{"parameter_id": "model_type","categorical_value_spec": {"values": ["nn"]}}] worker_pool_specs_override_json: JSON worker pool specs. E.g., [{"machine_spec": {"machine_type": "n1-standard-16"}},{},{},{"machine_spec": {"machine_type": "n1-standard-16"}}] - reduce_search_space_mode: The reduce search space mode. Possible - values: "regular" (default), "minimal", "full". + reduce_search_space_mode: The reduce search space mode. Possible values: "regular" (default), "minimal", "full". num_selected_trials: Number of selected trials. The number of weak learners in the final model is 5 * num_selected_trials. num_selected_features: Number of selected features. The number of features to learn in the NN models. deadline_hours: Number of hours the cross-validation trainer should run. @@ -110,11 +109,11 @@ def automl_tabular_stage_1_tuner( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "args": ["l2l_stage_1_tuner", "--transform_output_path=', transform_output.uri, '", "--training_docker_uri=', - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "--feature_selection_result_path=', feature_ranking.uri, '", "--disable_early_stopping=', diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stats_and_example_gen.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stats_and_example_gen.py index d213877c77..adfaac95e0 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stats_and_example_gen.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/stats_and_example_gen.py @@ -77,9 +77,7 @@ def tabular_stats_and_example_gen( target_column_name: The target column name. weight_column_name: The weight column name. prediction_type: The prediction type. Supported values: "classification", "regression". - optimization_objective: Objective function the model is optimizing towards. The training process creates a model that maximizes/minimizes the value of the objective function over the validation set. The supported optimization objectives depend on the prediction type. If the field is not set, a default objective function is used. - classification: "maximize-au-roc" (default) - Maximize the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall curve. "maximize-precision-at-recall" - Maximize precision for a specified recall value. "maximize-recall-at-precision" - Maximize recall for a specified precision value. classification (multi-class): "minimize-log-loss" (default) - Minimize log loss. - regression: "minimize-rmse" (default) - Minimize root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error (RMSLE). + optimization_objective: Objective function the model is optimizing towards. The training process creates a model that maximizes/minimizes the value of the objective function over the validation set. The supported optimization objectives depend on the prediction type. If the field is not set, a default objective function is used. classification: "maximize-au-roc" (default) - Maximize the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall curve. "maximize-precision-at-recall" - Maximize precision for a specified recall value. "maximize-recall-at-precision" - Maximize recall for a specified precision value. classification (multi-class): "minimize-log-loss" (default) - Minimize log loss. regression: "minimize-rmse" (default) - Minimize root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error (RMSLE). optimization_objective_recall_value: Required when optimization_objective is "maximize-precision-at-recall". Must be between 0 and 1, inclusive. optimization_objective_precision_value: Required when optimization_objective is "maximize-recall-at-precision". Must be between 0 and 1, inclusive. transformations: Quote escaped JSON string for transformations. Each transformation will apply transform function to given input column. And the result will be used for training. When creating transformation for BigQuery Struct column, the column should be flattened using "." as the delimiter. @@ -87,8 +85,7 @@ def tabular_stats_and_example_gen( dataflow_machine_type: The machine type used for dataflow jobs. If not set, default to n1-standard-16. dataflow_max_num_workers: The number of workers to run the dataflow job. If not set, default to 25. dataflow_disk_size_gb: The disk size, in gigabytes, to use on each Dataflow worker instance. If not set, default to 40. - dataflow_subnetwork: Dataflow's fully qualified subnetwork name, when empty the default subnetwork will be used. More - details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications + dataflow_subnetwork: Dataflow's fully qualified subnetwork name, when empty the default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications dataflow_use_public_ips: Specifies whether Dataflow workers use public IP addresses. dataflow_service_account: Custom service account to run dataflow jobs. encryption_spec_key_name: Customer-managed encryption key. @@ -139,7 +136,7 @@ def tabular_stats_and_example_gen( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', '", "args": ["stats_generator",', '"--train_spec={\\"prediction_type\\": \\"', prediction_type, @@ -218,7 +215,7 @@ def tabular_stats_and_example_gen( ), dataflow_max_num_workers, '", "--dataflow_worker_container_image=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125', '", "--dataflow_machine_type=', dataflow_machine_type, '", "--dataflow_disk_size_gb=', diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/training_configurator_and_validator.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/training_configurator_and_validator.py index 4cd8af17e9..2b0d803d99 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/training_configurator_and_validator.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/training_configurator_and_validator.py @@ -65,16 +65,14 @@ def training_configurator_and_validator( target_column: Target column of input data. weight_column: Weight column of input data. prediction_type: Model prediction type. One of "classification", "regression", "time_series". - optimization_objective: Objective function the model is optimizing towards. The training process creates a model that maximizes/minimizes the value of the objective function over the validation set. The supported optimization objectives depend on the prediction type. If the field is not set, a default objective function is used. - classification: "maximize-au-roc" (default) - Maximize the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall curve. "maximize-precision-at-recall" - Maximize precision for a specified recall value. "maximize-recall-at-precision" - Maximize recall for a specified precision value. classification (multi-class): "minimize-log-loss" (default) - Minimize log loss. - regression: "minimize-rmse" (default) - Minimize root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error (RMSLE). + optimization_objective: Objective function the model is optimizing towards. The training process creates a model that maximizes/minimizes the value of the objective function over the validation set. The supported optimization objectives depend on the prediction type. If the field is not set, a default objective function is used. classification: "maximize-au-roc" (default) - Maximize the area under the receiver operating characteristic (ROC) curve. "minimize-log-loss" - Minimize log loss. "maximize-au-prc" - Maximize the area under the precision-recall curve. "maximize-precision-at-recall" - Maximize precision for a specified recall value. "maximize-recall-at-precision" - Maximize recall for a specified precision value. classification (multi-class): "minimize-log-loss" (default) - Minimize log loss. regression: "minimize-rmse" (default) - Minimize root-mean-squared error (RMSE). "minimize-mae" - Minimize mean-absolute error (MAE). "minimize-rmsle" - Minimize root-mean-squared log error (RMSLE). optimization_objective_recall_value: Required when optimization_objective is "maximize-precision-at-recall". Must be between 0 and 1, inclusive. optimization_objective_precision_value: Required when optimization_objective is "maximize-recall-at-precision". Must be between 0 and 1, inclusive. run_evaluation: Whether we are running evaluation in the training pipeline. run_distill: Whether the distillation should be applied to the training. - enable_probabilistic_inference: If probabilistic inference is enabled, the model will fit a distribution that captures the uncertainty of a prediction. At inference time, the predictive distribution is used to make a point prediction that minimizes the optimization objective. For example, the mean of a predictive distribution is the point prediction that minimizes RMSE loss. If quantiles are specified, then the quantiles of the distribution are also returned. + enable_probabilistic_inference: If probabilistic inference is enabled, the model will fit a distribution that captures the uncertainty of a prediction. At inference time, the predictive distribution is used to make a point prediction that minimizes the optimization objective. For example, the mean of a predictive distribution is the point prediction that minimizes RMSE loss. If quantiles are specified, then the quantiles of the distribution are also returned. time_series_identifier_column: [Deprecated] The time series identifier column. Used by forecasting only. Raises exception if used - use the "time_series_identifier_column" field instead. - time_series_identifier_columns: The list of time series identifier columns. Used by forecasting only. + time_series_identifier_columns: The list of time series identifier columns. Used by forecasting only. time_column: The column that indicates the time. Used by forecasting only. time_series_attribute_columns: The column names of the time series attributes. available_at_forecast_columns: The names of the columns that are available at forecast time. @@ -97,7 +95,7 @@ def training_configurator_and_validator( # fmt: on return dsl.ContainerSpec( - image='us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20231029_0125', + image='us-docker.pkg.dev/vertex-ai/automl-tabular/feature-transform-engine:20240119_0125', command=[], args=[ 'training_configurator_and_validator', diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/transform.py b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/transform.py index d84da3ffef..230c63fad9 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/transform.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/automl/tabular/transform.py @@ -62,8 +62,7 @@ def automl_tabular_transform( dataflow_machine_type: The machine type used for dataflow jobs. If not set, default to n1-standard-16. dataflow_max_num_workers: The number of workers to run the dataflow job. If not set, default to 25. dataflow_disk_size_gb: The disk size, in gigabytes, to use on each Dataflow worker instance. If not set, default to 40. - dataflow_subnetwork: Dataflow's fully qualified subnetwork name, when empty the default subnetwork will be used. More - details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications + dataflow_subnetwork: Dataflow's fully qualified subnetwork name, when empty the default subnetwork will be used. More details: https://cloud.google.com/dataflow/docs/guides/specifying-networks#example_network_and_subnetwork_specifications dataflow_use_public_ips: Specifies whether Dataflow workers use public IP addresses. dataflow_service_account: Custom service account to run dataflow jobs. encryption_spec_key_name: Customer-managed encryption key. @@ -109,7 +108,7 @@ def automl_tabular_transform( ' 1, "machine_spec": {"machine_type": "n1-standard-8"},' ' "container_spec": {"image_uri":"' ), - 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20231029_0125', + 'us-docker.pkg.dev/vertex-ai-restricted/automl-tabular/training:20240119_0125', ( '", "args": ["transform", "--is_mp=true",' ' "--transform_output_artifact_path=' @@ -168,7 +167,7 @@ def automl_tabular_transform( '", "--dataflow_machine_type=', dataflow_machine_type, '", "--dataflow_worker_container_image=', - 'us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20231029_0125', + 'us-docker.pkg.dev/vertex-ai/automl-tabular/dataflow-worker:20240119_0125', '", "--dataflow_disk_size_gb=', dataflow_disk_size_gb, '", "--dataflow_subnetwork_fully_qualified=', diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/model/__init__.py b/components/google-cloud/google_cloud_pipeline_components/v1/model/__init__.py index 2295c68d53..593e2110f0 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/model/__init__.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/model/__init__.py @@ -1,4 +1,4 @@ -# Copyright 2023 The Kubeflow Authors. All Rights Reserved. +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -17,10 +17,12 @@ from google_cloud_pipeline_components.v1.model.delete_model.component import model_delete as ModelDeleteOp from google_cloud_pipeline_components.v1.model.export_model.component import model_export as ModelExportOp +from google_cloud_pipeline_components.v1.model.get_model.component import model_get as ModelGetOp from google_cloud_pipeline_components.v1.model.upload_model.component import model_upload as ModelUploadOp __all__ = [ 'ModelExportOp', 'ModelUploadOp', 'ModelDeleteOp', + 'ModelGetOp', ] diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/model/get_model/__init__.py b/components/google-cloud/google_cloud_pipeline_components/v1/model/get_model/__init__.py new file mode 100644 index 0000000000..e56f1e7480 --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/v1/model/get_model/__init__.py @@ -0,0 +1,14 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Google Cloud Pipeline Get Vertex Model Component.""" diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/model/get_model/component.py b/components/google-cloud/google_cloud_pipeline_components/v1/model/get_model/component.py new file mode 100644 index 0000000000..5583664c0a --- /dev/null +++ b/components/google-cloud/google_cloud_pipeline_components/v1/model/get_model/component.py @@ -0,0 +1,58 @@ +# Copyright 2024 The Kubeflow Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from google_cloud_pipeline_components import _image +from google_cloud_pipeline_components import _placeholders +from google_cloud_pipeline_components.types.artifact_types import VertexModel +from kfp import dsl + + +@dsl.container_component +def model_get( + model: dsl.Output[VertexModel], + model_name: str, + project: str = _placeholders.PROJECT_ID_PLACEHOLDER, + location: str = 'us-central1', +): + # fmt: off + """Gets a model artifact based on the model name of an existing Vertex model. + + Args: + project: Project from which to get the VertexModel. Defaults to the project in which the PipelineJob is run. + model_name: Vertex model resource name in the format of `projects/{project}/locations/{location}/models/{model}` or `projects/{project}/locations/{location}/models/{model}@{model_version_id or model_version_alias}`. If no version ID or alias is specified, the "default" version will be returned. + location: Location from which to get the VertexModel. Defaults to `us-central1`. + + Returns: + model: Artifact of the Vertex Model. + """ + # fmt: on + return dsl.ContainerSpec( + image=_image.GCPC_IMAGE_TAG, + command=[ + 'python3', + '-u', + '-m', + 'google_cloud_pipeline_components.container.v1.model.get_model.launcher', + ], + args=[ + '--project', + project, + '--location', + location, + '--model_name', + model_name, + '--executor_input', + '{{$}}', + ], + ) diff --git a/components/google-cloud/google_cloud_pipeline_components/v1/model/upload_model/component.py b/components/google-cloud/google_cloud_pipeline_components/v1/model/upload_model/component.py index 7f9397b80a..752f639cbf 100644 --- a/components/google-cloud/google_cloud_pipeline_components/v1/model/upload_model/component.py +++ b/components/google-cloud/google_cloud_pipeline_components/v1/model/upload_model/component.py @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -from typing import Dict +from typing import Dict, List from google_cloud_pipeline_components import _image from google_cloud_pipeline_components import _placeholders @@ -39,6 +39,7 @@ def model_upload( unmanaged_container_model: Input[UnmanagedContainerModel] = None, explanation_metadata: Dict[str, str] = {}, explanation_parameters: Dict[str, str] = {}, + version_aliases: List[str] = [], labels: Dict[str, str] = {}, encryption_spec_key_name: str = '', project: str = _placeholders.PROJECT_ID_PLACEHOLDER, @@ -60,6 +61,7 @@ def model_upload( explanation_metadata: Metadata describing the Model's input and output for explanation. Both `explanation_metadata` and `explanation_parameters` must be passed together when used. [More information.](https://cloud.google.com/vertex-ai/docs/reference/rest/v1/ExplanationSpec#explanationmetadata) explanation_parameters: Parameters to configure explaining for Model's predictions. [More information.](https://cloud.google.com/vertex-ai/docs/reference/rest/v1/ExplanationSpec#ExplanationParameters) + version_aliases: User provided version aliases so that a model version can be referenced via alias (i.e. `projects/{project}/locations/{location}/models/{modelId}@{version_alias}` instead of auto-generated version id (i.e. `projects/{project}/locations/{location}/models/{modelId}@{versionId}`). The format is [a-z][a-zA-Z0-9-]{0,126}[a-z0-9] to distinguish from versionId. A default version alias will be created for the first version of the model, and there must be exactly one default version alias for a model. encryption_spec_key_name: Customer-managed encryption key spec for a Model. If set, this Model and all sub-resources of this Model will be secured by this key. Has the form: `projects/my-project/locations/my-location/keyRings/my-kr/cryptoKeys/my-key`. The key needs to be in the same region as where the compute resource is created. labels: The labels with user-defined metadata to organize your model. Label keys and values can be no longer than 64 characters (Unicode codepoints), can only contain lowercase letters, numeric characters, underscores and dashes. International characters are allowed. See https://goo.gl/xmQnxf for more information and examples of labels. project: Project to upload this Model to. Defaults to the project in which the PipelineJob is run. @@ -98,6 +100,8 @@ def model_upload( ', "encryption_spec": {"kms_key_name":"', encryption_spec_key_name, '"}', + ', "version_aliases": ', + version_aliases, ', "labels": ', labels, ', "pipeline_job": "', diff --git a/components/google-cloud/google_cloud_pipeline_components/version.py b/components/google-cloud/google_cloud_pipeline_components/version.py index 9cea42fd12..01aab11847 100644 --- a/components/google-cloud/google_cloud_pipeline_components/version.py +++ b/components/google-cloud/google_cloud_pipeline_components/version.py @@ -13,4 +13,4 @@ # limitations under the License. """Google Cloud Pipeline Components version.""" -__version__ = "2.8.0" +__version__ = "2.9.0" diff --git a/components/google-cloud/setup.py b/components/google-cloud/setup.py index af54486684..7f288ff938 100644 --- a/components/google-cloud/setup.py +++ b/components/google-cloud/setup.py @@ -64,15 +64,15 @@ # related to protobuf # second list of deps are true dependencies for building the site "docs": [ - "protobuf<4.0.0dev,>=3.19.0", + "protobuf>=4.21.1,<5", "grpcio-status<=1.47.0", ] + [ "commonmark==0.9.1", "autodocsumm==0.2.9", - "sphinx==5.0.2", + "sphinx>=5.0.2,<6.0.0", "sphinx-immaterial==0.9.0", - "sphinx-rtd-theme==1.0.0", - "m2r2==0.3.3", + "sphinx-rtd-theme==2.0.0", + "m2r2==0.3.3.post2", "sphinx-notfound-page==0.8.3", ], }, @@ -82,9 +82,9 @@ # Pin google-api-core version for the bug fixing in 1.31.5 # https://github.com/googleapis/python-api-core/releases/tag/v1.31.5 "google-api-core>=1.31.5,<3.0.0dev,!=2.0.*,!=2.1.*,!=2.2.*,!=2.3.0", - "kfp>=2.0.0b10,<=2.4.0", + "kfp>=2.6.0,<=2.6.0", "google-cloud-aiplatform>=1.14.0,<2", - "Jinja2==3.1.2", + "Jinja2>=3.1.2,<4", ], project_urls={ "User Documentation": "https://cloud.google.com/vertex-ai/docs/pipelines/components-introduction", diff --git a/docs/conf.py b/docs/conf.py index cf6d7398dc..7d9aaa46a4 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -132,12 +132,19 @@ True, 'version_info': [ # need to use the sdk- prefix to avoid conflict with the BE's GitHub release tags + { + 'version': + 'https://kubeflow-pipelines.readthedocs.io/en/sdk-2.7.0/', + 'title': + '2.7.0', + 'aliases': ['stable'], + }, { 'version': 'https://kubeflow-pipelines.readthedocs.io/en/sdk-2.6.0/', 'title': '2.6.0', - 'aliases': ['stable'], + 'aliases': [], }, { 'version': diff --git a/docs/source/kfp.rst b/docs/source/kfp.rst index 01df01fbc9..634bdbd4b2 100644 --- a/docs/source/kfp.rst +++ b/docs/source/kfp.rst @@ -9,3 +9,4 @@ API Reference components client registry + local diff --git a/docs/source/local.rst b/docs/source/local.rst new file mode 100644 index 0000000000..6cbf817d76 --- /dev/null +++ b/docs/source/local.rst @@ -0,0 +1,4 @@ +kfp.local +========================== + +.. automodule:: kfp.local diff --git a/go.mod b/go.mod index 90fb9c04bb..f8bd701767 100644 --- a/go.mod +++ b/go.mod @@ -32,7 +32,7 @@ require ( github.com/kubeflow/kfp-tekton/tekton-catalog/tekton-exithandler v0.0.0-20231127195001-a75d4b3711ff github.com/kubeflow/kfp-tekton/tekton-catalog/tekton-kfptask v0.0.0-20231127195001-a75d4b3711ff github.com/kubeflow/pipelines/api v0.0.0-20231027040853-58ce09e07d03 - github.com/kubeflow/pipelines/kubernetes_platform v0.0.0-20240111221413-aac4408237df + github.com/kubeflow/pipelines/kubernetes_platform v0.0.0-20240207171236-f51dc39614e4 github.com/kubeflow/pipelines/third_party/ml-metadata v0.0.0-20230810215105-e1f0c010f800 github.com/lestrrat-go/strftime v1.0.4 github.com/mattn/go-sqlite3 v1.14.16 diff --git a/go.sum b/go.sum index 3deaa5ffd8..6e39439298 100644 --- a/go.sum +++ b/go.sum @@ -2190,8 +2190,8 @@ github.com/kubeflow/kfp-tekton/tekton-catalog/tekton-kfptask v0.0.0-202311271950 github.com/kubeflow/kfp-tekton/tekton-catalog/tekton-kfptask v0.0.0-20231127195001-a75d4b3711ff/go.mod h1:lAFdPugzj3bcAXyN3+8y0NByidZ88zwGxMc+gdc8cHw= github.com/kubeflow/pipelines/api v0.0.0-20231027040853-58ce09e07d03 h1:reL3LbkRIozBkKSUYjtQFV2kVC1R4WHG9FrTClRT1FY= github.com/kubeflow/pipelines/api v0.0.0-20231027040853-58ce09e07d03/go.mod h1:T7TOQB36gGe97yUdfVAnYK5uuT0+uQbLNHDUHxYkmE4= -github.com/kubeflow/pipelines/kubernetes_platform v0.0.0-20240111221413-aac4408237df h1:hskr5e/bAKNi8Bc15lG9uZ28q36v/y8qbKkDDix9NPo= -github.com/kubeflow/pipelines/kubernetes_platform v0.0.0-20240111221413-aac4408237df/go.mod h1:CJkKr356RlpZP/gQRuHf3Myrn1qJtoUVe4EMCmtwarg= +github.com/kubeflow/pipelines/kubernetes_platform v0.0.0-20240207171236-f51dc39614e4 h1:4WGf/JTH2Pks3A1fru2lk2u8gO/MR3g7tPJC7OXhAzk= +github.com/kubeflow/pipelines/kubernetes_platform v0.0.0-20240207171236-f51dc39614e4/go.mod h1:CJkKr356RlpZP/gQRuHf3Myrn1qJtoUVe4EMCmtwarg= github.com/kubeflow/pipelines/third_party/ml-metadata v0.0.0-20230810215105-e1f0c010f800 h1:YAW+X9xCW8Yq5tQaBBQaLTNU9CJj8Nr7lx1+k66ZHJ0= github.com/kubeflow/pipelines/third_party/ml-metadata v0.0.0-20230810215105-e1f0c010f800/go.mod h1:chIDffBaVQ/asNl1pTTdbAymYcuBKf8BR3YtSP+3FEU= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= diff --git a/kubernetes_platform/go/kubernetesplatform/kubernetes_executor_config.pb.go b/kubernetes_platform/go/kubernetesplatform/kubernetes_executor_config.pb.go index c536b54152..ef9a6d1bee 100644 --- a/kubernetes_platform/go/kubernetesplatform/kubernetes_executor_config.pb.go +++ b/kubernetes_platform/go/kubernetesplatform/kubernetes_executor_config.pb.go @@ -40,11 +40,17 @@ type KubernetesExecutorConfig struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - SecretAsVolume []*SecretAsVolume `protobuf:"bytes,1,rep,name=secret_as_volume,json=secretAsVolume,proto3" json:"secret_as_volume,omitempty"` - SecretAsEnv []*SecretAsEnv `protobuf:"bytes,2,rep,name=secret_as_env,json=secretAsEnv,proto3" json:"secret_as_env,omitempty"` - PvcMount []*PvcMount `protobuf:"bytes,3,rep,name=pvc_mount,json=pvcMount,proto3" json:"pvc_mount,omitempty"` - NodeSelector *NodeSelector `protobuf:"bytes,4,opt,name=node_selector,json=nodeSelector,proto3" json:"node_selector,omitempty"` - PodMetadata *PodMetadata `protobuf:"bytes,5,opt,name=pod_metadata,json=podMetadata,proto3" json:"pod_metadata,omitempty"` + SecretAsVolume []*SecretAsVolume `protobuf:"bytes,1,rep,name=secret_as_volume,json=secretAsVolume,proto3" json:"secret_as_volume,omitempty"` + SecretAsEnv []*SecretAsEnv `protobuf:"bytes,2,rep,name=secret_as_env,json=secretAsEnv,proto3" json:"secret_as_env,omitempty"` + PvcMount []*PvcMount `protobuf:"bytes,3,rep,name=pvc_mount,json=pvcMount,proto3" json:"pvc_mount,omitempty"` + NodeSelector *NodeSelector `protobuf:"bytes,4,opt,name=node_selector,json=nodeSelector,proto3" json:"node_selector,omitempty"` + PodMetadata *PodMetadata `protobuf:"bytes,5,opt,name=pod_metadata,json=podMetadata,proto3" json:"pod_metadata,omitempty"` + ImagePullSecret []*ImagePullSecret `protobuf:"bytes,6,rep,name=image_pull_secret,json=imagePullSecret,proto3" json:"image_pull_secret,omitempty"` + // One of Always, Never, IfNotPresent. + ImagePullPolicy string `protobuf:"bytes,7,opt,name=image_pull_policy,json=imagePullPolicy,proto3" json:"image_pull_policy,omitempty"` + ConfigMapAsVolume []*ConfigMapAsVolume `protobuf:"bytes,8,rep,name=config_map_as_volume,json=configMapAsVolume,proto3" json:"config_map_as_volume,omitempty"` + ConfigMapAsEnv []*ConfigMapAsEnv `protobuf:"bytes,9,rep,name=config_map_as_env,json=configMapAsEnv,proto3" json:"config_map_as_env,omitempty"` + ActiveDeadlineSeconds int64 `protobuf:"varint,10,opt,name=active_deadline_seconds,json=activeDeadlineSeconds,proto3" json:"active_deadline_seconds,omitempty"` } func (x *KubernetesExecutorConfig) Reset() { @@ -114,6 +120,41 @@ func (x *KubernetesExecutorConfig) GetPodMetadata() *PodMetadata { return nil } +func (x *KubernetesExecutorConfig) GetImagePullSecret() []*ImagePullSecret { + if x != nil { + return x.ImagePullSecret + } + return nil +} + +func (x *KubernetesExecutorConfig) GetImagePullPolicy() string { + if x != nil { + return x.ImagePullPolicy + } + return "" +} + +func (x *KubernetesExecutorConfig) GetConfigMapAsVolume() []*ConfigMapAsVolume { + if x != nil { + return x.ConfigMapAsVolume + } + return nil +} + +func (x *KubernetesExecutorConfig) GetConfigMapAsEnv() []*ConfigMapAsEnv { + if x != nil { + return x.ConfigMapAsEnv + } + return nil +} + +func (x *KubernetesExecutorConfig) GetActiveDeadlineSeconds() int64 { + if x != nil { + return x.ActiveDeadlineSeconds + } + return 0 +} + type SecretAsVolume struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -739,6 +780,167 @@ func (x *PodMetadata) GetAnnotations() map[string]string { return nil } +type ConfigMapAsVolume struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Name of the ConfigMap. + ConfigMapName string `protobuf:"bytes,1,opt,name=config_map_name,json=configMapName,proto3" json:"config_map_name,omitempty"` + // Container path to mount the ConfigMap data. + MountPath string `protobuf:"bytes,2,opt,name=mount_path,json=mountPath,proto3" json:"mount_path,omitempty"` +} + +func (x *ConfigMapAsVolume) Reset() { + *x = ConfigMapAsVolume{} + if protoimpl.UnsafeEnabled { + mi := &file_kubernetes_executor_config_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ConfigMapAsVolume) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConfigMapAsVolume) ProtoMessage() {} + +func (x *ConfigMapAsVolume) ProtoReflect() protoreflect.Message { + mi := &file_kubernetes_executor_config_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ConfigMapAsVolume.ProtoReflect.Descriptor instead. +func (*ConfigMapAsVolume) Descriptor() ([]byte, []int) { + return file_kubernetes_executor_config_proto_rawDescGZIP(), []int{9} +} + +func (x *ConfigMapAsVolume) GetConfigMapName() string { + if x != nil { + return x.ConfigMapName + } + return "" +} + +func (x *ConfigMapAsVolume) GetMountPath() string { + if x != nil { + return x.MountPath + } + return "" +} + +type ConfigMapAsEnv struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Name of the ConfigMap. + ConfigMapName string `protobuf:"bytes,1,opt,name=config_map_name,json=configMapName,proto3" json:"config_map_name,omitempty"` + KeyToEnv []*ConfigMapAsEnv_ConfigMapKeyToEnvMap `protobuf:"bytes,2,rep,name=key_to_env,json=keyToEnv,proto3" json:"key_to_env,omitempty"` +} + +func (x *ConfigMapAsEnv) Reset() { + *x = ConfigMapAsEnv{} + if protoimpl.UnsafeEnabled { + mi := &file_kubernetes_executor_config_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ConfigMapAsEnv) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConfigMapAsEnv) ProtoMessage() {} + +func (x *ConfigMapAsEnv) ProtoReflect() protoreflect.Message { + mi := &file_kubernetes_executor_config_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ConfigMapAsEnv.ProtoReflect.Descriptor instead. +func (*ConfigMapAsEnv) Descriptor() ([]byte, []int) { + return file_kubernetes_executor_config_proto_rawDescGZIP(), []int{10} +} + +func (x *ConfigMapAsEnv) GetConfigMapName() string { + if x != nil { + return x.ConfigMapName + } + return "" +} + +func (x *ConfigMapAsEnv) GetKeyToEnv() []*ConfigMapAsEnv_ConfigMapKeyToEnvMap { + if x != nil { + return x.KeyToEnv + } + return nil +} + +type ImagePullSecret struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Name of the image pull secret. + SecretName string `protobuf:"bytes,1,opt,name=secret_name,json=secretName,proto3" json:"secret_name,omitempty"` +} + +func (x *ImagePullSecret) Reset() { + *x = ImagePullSecret{} + if protoimpl.UnsafeEnabled { + mi := &file_kubernetes_executor_config_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImagePullSecret) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImagePullSecret) ProtoMessage() {} + +func (x *ImagePullSecret) ProtoReflect() protoreflect.Message { + mi := &file_kubernetes_executor_config_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ImagePullSecret.ProtoReflect.Descriptor instead. +func (*ImagePullSecret) Descriptor() ([]byte, []int) { + return file_kubernetes_executor_config_proto_rawDescGZIP(), []int{11} +} + +func (x *ImagePullSecret) GetSecretName() string { + if x != nil { + return x.SecretName + } + return "" +} + type SecretAsEnv_SecretKeyToEnvMap struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -753,7 +955,7 @@ type SecretAsEnv_SecretKeyToEnvMap struct { func (x *SecretAsEnv_SecretKeyToEnvMap) Reset() { *x = SecretAsEnv_SecretKeyToEnvMap{} if protoimpl.UnsafeEnabled { - mi := &file_kubernetes_executor_config_proto_msgTypes[9] + mi := &file_kubernetes_executor_config_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -766,7 +968,7 @@ func (x *SecretAsEnv_SecretKeyToEnvMap) String() string { func (*SecretAsEnv_SecretKeyToEnvMap) ProtoMessage() {} func (x *SecretAsEnv_SecretKeyToEnvMap) ProtoReflect() protoreflect.Message { - mi := &file_kubernetes_executor_config_proto_msgTypes[9] + mi := &file_kubernetes_executor_config_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -796,6 +998,63 @@ func (x *SecretAsEnv_SecretKeyToEnvMap) GetEnvVar() string { return "" } +type ConfigMapAsEnv_ConfigMapKeyToEnvMap struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Corresponds to a key of the ConfigMap. + ConfigMapKey string `protobuf:"bytes,1,opt,name=config_map_key,json=configMapKey,proto3" json:"config_map_key,omitempty"` + // Env var to which configmap_key's data should be set. + EnvVar string `protobuf:"bytes,2,opt,name=env_var,json=envVar,proto3" json:"env_var,omitempty"` +} + +func (x *ConfigMapAsEnv_ConfigMapKeyToEnvMap) Reset() { + *x = ConfigMapAsEnv_ConfigMapKeyToEnvMap{} + if protoimpl.UnsafeEnabled { + mi := &file_kubernetes_executor_config_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ConfigMapAsEnv_ConfigMapKeyToEnvMap) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConfigMapAsEnv_ConfigMapKeyToEnvMap) ProtoMessage() {} + +func (x *ConfigMapAsEnv_ConfigMapKeyToEnvMap) ProtoReflect() protoreflect.Message { + mi := &file_kubernetes_executor_config_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ConfigMapAsEnv_ConfigMapKeyToEnvMap.ProtoReflect.Descriptor instead. +func (*ConfigMapAsEnv_ConfigMapKeyToEnvMap) Descriptor() ([]byte, []int) { + return file_kubernetes_executor_config_proto_rawDescGZIP(), []int{10, 0} +} + +func (x *ConfigMapAsEnv_ConfigMapKeyToEnvMap) GetConfigMapKey() string { + if x != nil { + return x.ConfigMapKey + } + return "" +} + +func (x *ConfigMapAsEnv_ConfigMapKeyToEnvMap) GetEnvVar() string { + if x != nil { + return x.EnvVar + } + return "" +} + var File_kubernetes_executor_config_proto protoreflect.FileDescriptor var file_kubernetes_executor_config_proto_rawDesc = []byte{ @@ -804,7 +1063,7 @@ var file_kubernetes_executor_config_proto_rawDesc = []byte{ 0x74, 0x6f, 0x12, 0x0e, 0x6b, 0x66, 0x70, 0x5f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 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, 0xdf, 0x02, 0x0a, 0x18, 0x4b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x45, + 0x22, 0xaf, 0x05, 0x0a, 0x18, 0x4b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x48, 0x0a, 0x10, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, 0x61, 0x73, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6b, 0x66, 0x70, 0x5f, 0x6b, 0x75, @@ -826,7 +1085,28 @@ var file_kubernetes_executor_config_proto_rawDesc = []byte{ 0x74, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6b, 0x66, 0x70, 0x5f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x2e, 0x50, 0x6f, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x0b, 0x70, 0x6f, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x22, 0x50, 0x0a, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x73, 0x56, 0x6f, + 0x74, 0x61, 0x12, 0x4b, 0x0a, 0x11, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x75, 0x6c, 0x6c, + 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, + 0x6b, 0x66, 0x70, 0x5f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x2e, 0x49, + 0x6d, 0x61, 0x67, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x52, 0x0f, + 0x69, 0x6d, 0x61, 0x67, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x12, + 0x2a, 0x0a, 0x11, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x75, 0x6c, 0x6c, 0x5f, 0x70, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x69, 0x6d, 0x61, 0x67, + 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x52, 0x0a, 0x14, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6d, 0x61, 0x70, 0x5f, 0x61, 0x73, 0x5f, 0x76, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6b, 0x66, 0x70, 0x5f, + 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x4d, 0x61, 0x70, 0x41, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x11, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x4d, 0x61, 0x70, 0x41, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, + 0x49, 0x0a, 0x11, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6d, 0x61, 0x70, 0x5f, 0x61, 0x73, + 0x5f, 0x65, 0x6e, 0x76, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6b, 0x66, 0x70, + 0x5f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x2e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x4d, 0x61, 0x70, 0x41, 0x73, 0x45, 0x6e, 0x76, 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x4d, 0x61, 0x70, 0x41, 0x73, 0x45, 0x6e, 0x76, 0x12, 0x36, 0x0a, 0x17, 0x61, 0x63, + 0x74, 0x69, 0x76, 0x65, 0x5f, 0x64, 0x65, 0x61, 0x64, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x73, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x61, 0x63, 0x74, + 0x69, 0x76, 0x65, 0x44, 0x65, 0x61, 0x64, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x65, 0x63, 0x6f, 0x6e, + 0x64, 0x73, 0x22, 0x50, 0x0a, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x70, @@ -927,13 +1207,36 @@ var file_kubernetes_executor_config_proto_rawDesc = []byte{ 0x3e, 0x0a, 0x10, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x42, - 0x49, 0x5a, 0x47, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6b, 0x75, - 0x62, 0x65, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, - 0x2f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x5f, 0x70, 0x6c, 0x61, 0x74, - 0x66, 0x6f, 0x72, 0x6d, 0x2f, 0x67, 0x6f, 0x2f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, - 0x65, 0x73, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0x5a, 0x0a, 0x11, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4d, 0x61, 0x70, 0x41, 0x73, 0x56, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6d, + 0x61, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4d, 0x61, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, + 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x50, 0x61, 0x74, 0x68, 0x22, 0xe2, 0x01, 0x0a, 0x0e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4d, 0x61, 0x70, 0x41, 0x73, 0x45, 0x6e, 0x76, 0x12, 0x26, + 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6d, 0x61, 0x70, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4d, + 0x61, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x51, 0x0a, 0x0a, 0x6b, 0x65, 0x79, 0x5f, 0x74, 0x6f, + 0x5f, 0x65, 0x6e, 0x76, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6b, 0x66, 0x70, + 0x5f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x2e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x4d, 0x61, 0x70, 0x41, 0x73, 0x45, 0x6e, 0x76, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x4d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x54, 0x6f, 0x45, 0x6e, 0x76, 0x4d, 0x61, 0x70, 0x52, + 0x08, 0x6b, 0x65, 0x79, 0x54, 0x6f, 0x45, 0x6e, 0x76, 0x1a, 0x55, 0x0a, 0x14, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x4d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x54, 0x6f, 0x45, 0x6e, 0x76, 0x4d, 0x61, + 0x70, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6d, 0x61, 0x70, 0x5f, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x4d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x12, 0x17, 0x0a, 0x07, 0x65, 0x6e, 0x76, 0x5f, 0x76, + 0x61, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x65, 0x6e, 0x76, 0x56, 0x61, 0x72, + 0x22, 0x32, 0x0a, 0x0f, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x53, 0x65, 0x63, + 0x72, 0x65, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, + 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x49, 0x5a, 0x47, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6b, 0x75, 0x62, 0x65, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, + 0x5f, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x2f, 0x67, 0x6f, 0x2f, 0x6b, 0x75, 0x62, + 0x65, 0x72, 0x6e, 0x65, 0x74, 0x65, 0x73, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -948,7 +1251,7 @@ func file_kubernetes_executor_config_proto_rawDescGZIP() []byte { return file_kubernetes_executor_config_proto_rawDescData } -var file_kubernetes_executor_config_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_kubernetes_executor_config_proto_msgTypes = make([]protoimpl.MessageInfo, 17) var file_kubernetes_executor_config_proto_goTypes = []interface{}{ (*KubernetesExecutorConfig)(nil), // 0: kfp_kubernetes.KubernetesExecutorConfig (*SecretAsVolume)(nil), // 1: kfp_kubernetes.SecretAsVolume @@ -959,11 +1262,15 @@ var file_kubernetes_executor_config_proto_goTypes = []interface{}{ (*DeletePvc)(nil), // 6: kfp_kubernetes.DeletePvc (*NodeSelector)(nil), // 7: kfp_kubernetes.NodeSelector (*PodMetadata)(nil), // 8: kfp_kubernetes.PodMetadata - (*SecretAsEnv_SecretKeyToEnvMap)(nil), // 9: kfp_kubernetes.SecretAsEnv.SecretKeyToEnvMap - nil, // 10: kfp_kubernetes.NodeSelector.LabelsEntry - nil, // 11: kfp_kubernetes.PodMetadata.LabelsEntry - nil, // 12: kfp_kubernetes.PodMetadata.AnnotationsEntry - (*structpb.Struct)(nil), // 13: google.protobuf.Struct + (*ConfigMapAsVolume)(nil), // 9: kfp_kubernetes.ConfigMapAsVolume + (*ConfigMapAsEnv)(nil), // 10: kfp_kubernetes.ConfigMapAsEnv + (*ImagePullSecret)(nil), // 11: kfp_kubernetes.ImagePullSecret + (*SecretAsEnv_SecretKeyToEnvMap)(nil), // 12: kfp_kubernetes.SecretAsEnv.SecretKeyToEnvMap + nil, // 13: kfp_kubernetes.NodeSelector.LabelsEntry + nil, // 14: kfp_kubernetes.PodMetadata.LabelsEntry + nil, // 15: kfp_kubernetes.PodMetadata.AnnotationsEntry + (*ConfigMapAsEnv_ConfigMapKeyToEnvMap)(nil), // 16: kfp_kubernetes.ConfigMapAsEnv.ConfigMapKeyToEnvMap + (*structpb.Struct)(nil), // 17: google.protobuf.Struct } var file_kubernetes_executor_config_proto_depIdxs = []int32{ 1, // 0: kfp_kubernetes.KubernetesExecutorConfig.secret_as_volume:type_name -> kfp_kubernetes.SecretAsVolume @@ -971,18 +1278,22 @@ var file_kubernetes_executor_config_proto_depIdxs = []int32{ 4, // 2: kfp_kubernetes.KubernetesExecutorConfig.pvc_mount:type_name -> kfp_kubernetes.PvcMount 7, // 3: kfp_kubernetes.KubernetesExecutorConfig.node_selector:type_name -> kfp_kubernetes.NodeSelector 8, // 4: kfp_kubernetes.KubernetesExecutorConfig.pod_metadata:type_name -> kfp_kubernetes.PodMetadata - 9, // 5: kfp_kubernetes.SecretAsEnv.key_to_env:type_name -> kfp_kubernetes.SecretAsEnv.SecretKeyToEnvMap - 3, // 6: kfp_kubernetes.PvcMount.task_output_parameter:type_name -> kfp_kubernetes.TaskOutputParameterSpec - 13, // 7: kfp_kubernetes.CreatePvc.annotations:type_name -> google.protobuf.Struct - 3, // 8: kfp_kubernetes.DeletePvc.task_output_parameter:type_name -> kfp_kubernetes.TaskOutputParameterSpec - 10, // 9: kfp_kubernetes.NodeSelector.labels:type_name -> kfp_kubernetes.NodeSelector.LabelsEntry - 11, // 10: kfp_kubernetes.PodMetadata.labels:type_name -> kfp_kubernetes.PodMetadata.LabelsEntry - 12, // 11: kfp_kubernetes.PodMetadata.annotations:type_name -> kfp_kubernetes.PodMetadata.AnnotationsEntry - 12, // [12:12] is the sub-list for method output_type - 12, // [12:12] is the sub-list for method input_type - 12, // [12:12] is the sub-list for extension type_name - 12, // [12:12] is the sub-list for extension extendee - 0, // [0:12] is the sub-list for field type_name + 11, // 5: kfp_kubernetes.KubernetesExecutorConfig.image_pull_secret:type_name -> kfp_kubernetes.ImagePullSecret + 9, // 6: kfp_kubernetes.KubernetesExecutorConfig.config_map_as_volume:type_name -> kfp_kubernetes.ConfigMapAsVolume + 10, // 7: kfp_kubernetes.KubernetesExecutorConfig.config_map_as_env:type_name -> kfp_kubernetes.ConfigMapAsEnv + 12, // 8: kfp_kubernetes.SecretAsEnv.key_to_env:type_name -> kfp_kubernetes.SecretAsEnv.SecretKeyToEnvMap + 3, // 9: kfp_kubernetes.PvcMount.task_output_parameter:type_name -> kfp_kubernetes.TaskOutputParameterSpec + 17, // 10: kfp_kubernetes.CreatePvc.annotations:type_name -> google.protobuf.Struct + 3, // 11: kfp_kubernetes.DeletePvc.task_output_parameter:type_name -> kfp_kubernetes.TaskOutputParameterSpec + 13, // 12: kfp_kubernetes.NodeSelector.labels:type_name -> kfp_kubernetes.NodeSelector.LabelsEntry + 14, // 13: kfp_kubernetes.PodMetadata.labels:type_name -> kfp_kubernetes.PodMetadata.LabelsEntry + 15, // 14: kfp_kubernetes.PodMetadata.annotations:type_name -> kfp_kubernetes.PodMetadata.AnnotationsEntry + 16, // 15: kfp_kubernetes.ConfigMapAsEnv.key_to_env:type_name -> kfp_kubernetes.ConfigMapAsEnv.ConfigMapKeyToEnvMap + 16, // [16:16] is the sub-list for method output_type + 16, // [16:16] is the sub-list for method input_type + 16, // [16:16] is the sub-list for extension type_name + 16, // [16:16] is the sub-list for extension extendee + 0, // [0:16] is the sub-list for field type_name } func init() { file_kubernetes_executor_config_proto_init() } @@ -1100,6 +1411,42 @@ func file_kubernetes_executor_config_proto_init() { } } file_kubernetes_executor_config_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ConfigMapAsVolume); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_kubernetes_executor_config_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ConfigMapAsEnv); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_kubernetes_executor_config_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ImagePullSecret); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_kubernetes_executor_config_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SecretAsEnv_SecretKeyToEnvMap); i { case 0: return &v.state @@ -1111,6 +1458,18 @@ func file_kubernetes_executor_config_proto_init() { return nil } } + file_kubernetes_executor_config_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ConfigMapAsEnv_ConfigMapKeyToEnvMap); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_kubernetes_executor_config_proto_msgTypes[4].OneofWrappers = []interface{}{ (*PvcMount_TaskOutputParameter)(nil), @@ -1132,7 +1491,7 @@ func file_kubernetes_executor_config_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_kubernetes_executor_config_proto_rawDesc, NumEnums: 0, - NumMessages: 13, + NumMessages: 17, NumExtensions: 0, NumServices: 0, }, diff --git a/kubernetes_platform/proto/kubernetes_executor_config.proto b/kubernetes_platform/proto/kubernetes_executor_config.proto index 8b215c8ae1..46bcc362cc 100644 --- a/kubernetes_platform/proto/kubernetes_executor_config.proto +++ b/kubernetes_platform/proto/kubernetes_executor_config.proto @@ -26,6 +26,12 @@ message KubernetesExecutorConfig { repeated PvcMount pvc_mount = 3; NodeSelector node_selector = 4; PodMetadata pod_metadata = 5; + repeated ImagePullSecret image_pull_secret = 6; + // One of Always, Never, IfNotPresent. + string image_pull_policy = 7; + repeated ConfigMapAsVolume config_map_as_volume = 8; + repeated ConfigMapAsEnv config_map_as_env = 9; + int64 active_deadline_seconds = 10; } message SecretAsVolume { @@ -122,3 +128,29 @@ message PodMetadata { map labels = 1; map annotations = 2; } + +message ConfigMapAsVolume { + // Name of the ConfigMap. + string config_map_name = 1; + // Container path to mount the ConfigMap data. + string mount_path = 2; +} + +message ConfigMapAsEnv { + // Name of the ConfigMap. + string config_map_name = 1; + + message ConfigMapKeyToEnvMap { + // Corresponds to a key of the ConfigMap. + string config_map_key = 1; + // Env var to which configmap_key's data should be set. + string env_var = 2; + } + + repeated ConfigMapKeyToEnvMap key_to_env = 2; +} + +message ImagePullSecret { + // Name of the image pull secret. + string secret_name = 1; +} diff --git a/kubernetes_platform/python/README.md b/kubernetes_platform/python/README.md index df575b196a..652ad93e63 100644 --- a/kubernetes_platform/python/README.md +++ b/kubernetes_platform/python/README.md @@ -102,3 +102,29 @@ def my_pipeline(): delete_pvc1 = kubernetes.DeletePVC( pvc_name=pvc1.outputs['name']).after(task2) ``` + +### Pod Metadata: Add pod labels and annotations to the container pod's definition +```python +from kfp import dsl +from kfp import kubernetes + + +@dsl.component +def comp(): + pass + + +@dsl.pipeline +def my_pipeline(): + task = comp() + kubernetes.add_pod_label( + task, + label_key='kubeflow.com/kfp', + label_value='pipeline-node', + ) + kubernetes.add_pod_annotation( + task, + annotation_key='run_id', + annotation_value='123456', + ) +``` \ No newline at end of file diff --git a/kubernetes_platform/python/create_release_branch.sh b/kubernetes_platform/python/create_release_branch.sh index d1a8312e37..bf48c5447e 100644 --- a/kubernetes_platform/python/create_release_branch.sh +++ b/kubernetes_platform/python/create_release_branch.sh @@ -49,5 +49,5 @@ else rm $REPO_ROOT/kubernetes_platform/.gitignore - echo "\nNext steps:\n\t- Add the version number to $PKG_ROOT/docs/conf.py\n\t- Add and commit the changes in this branch using 'git add $REPO_ROOT && git commit -m 'update for release' --no-verify'\n\t- Push branch using 'git push --set-upstream upstream $BRANCH_NAME'" + echo "\nNext steps:\n\tPush branch using 'git push --set-upstream upstream $BRANCH_NAME'" fi diff --git a/kubernetes_platform/python/docs/.readthedocs.yml b/kubernetes_platform/python/docs/.readthedocs.yml index bf39275e59..290660f2e7 100644 --- a/kubernetes_platform/python/docs/.readthedocs.yml +++ b/kubernetes_platform/python/docs/.readthedocs.yml @@ -2,8 +2,11 @@ version: 2 sphinx: configuration: kubernetes_platform/python/docs/conf.py +build: + os: ubuntu-22.04 + tools: + python: "3.8" python: - version: 3.7 install: - requirements: kubernetes_platform/python/docs/requirements.txt - method: pip diff --git a/kubernetes_platform/python/docs/conf.py b/kubernetes_platform/python/docs/conf.py index 237a8f2c7c..24d3f29b32 100644 --- a/kubernetes_platform/python/docs/conf.py +++ b/kubernetes_platform/python/docs/conf.py @@ -137,13 +137,29 @@ def decorator(func): }, 'version_dropdown': True, - 'version_info': [{ - 'version': - 'https://kfp-kubernetes.readthedocs.io/en/kfp-kubernetes-0.0.1/', - 'title': - '0.0.1', - 'aliases': ['stable'], - },], + 'version_info': [ + { + 'version': + 'https://kfp-kubernetes.readthedocs.io/en/kfp-kubernetes-1.1.0/', + 'title': + '1.1.0', + 'aliases': ['stable'], + }, + { + 'version': + 'https://kfp-kubernetes.readthedocs.io/en/kfp-kubernetes-1.0.0/', + 'title': + '1.0.0', + 'aliases': [], + }, + { + 'version': + 'https://kfp-kubernetes.readthedocs.io/en/kfp-kubernetes-0.0.1/', + 'title': + '0.0.1', + 'aliases': [], + }, + ], # "toc_title_is_page_title": True, } # Add any paths that contain templates here, relative to this directory. diff --git a/kubernetes_platform/python/kfp/kubernetes/__init__.py b/kubernetes_platform/python/kfp/kubernetes/__init__.py index 587e5132b0..322bf7a305 100644 --- a/kubernetes_platform/python/kfp/kubernetes/__init__.py +++ b/kubernetes_platform/python/kfp/kubernetes/__init__.py @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = '1.0.0' +__version__ = '1.1.0' __all__ = [ 'CreatePVC', @@ -21,11 +21,17 @@ 'use_secret_as_env', 'use_secret_as_volume', 'add_node_selector', + 'add_pod_label', + 'add_pod_annotation', + 'set_image_pull_secrets' ] +from kfp.kubernetes.pod_metadata import add_pod_label +from kfp.kubernetes.pod_metadata import add_pod_annotation from kfp.kubernetes.node_selector import add_node_selector from kfp.kubernetes.secret import use_secret_as_env from kfp.kubernetes.secret import use_secret_as_volume from kfp.kubernetes.volume import CreatePVC from kfp.kubernetes.volume import DeletePVC from kfp.kubernetes.volume import mount_pvc +from kfp.kubernetes.image import set_image_pull_secrets diff --git a/kubernetes_platform/python/kfp/kubernetes/image.py b/kubernetes_platform/python/kfp/kubernetes/image.py new file mode 100644 index 0000000000..e7e7853b83 --- /dev/null +++ b/kubernetes_platform/python/kfp/kubernetes/image.py @@ -0,0 +1,48 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from typing import List + +from google.protobuf import json_format +from kfp.dsl import PipelineTask +from kfp.kubernetes import common +from kfp.kubernetes import kubernetes_executor_config_pb2 as pb + + +def set_image_pull_secrets( + task: PipelineTask, + secret_names: List[str], +) -> PipelineTask: + """Set image pull secrets for a Kubernetes task. + + Args: + task: Pipeline task. + secret_names: List of image pull secret names. + + Returns: + Task object with updated image pull secret configuration. + """ + + msg = common.get_existing_kubernetes_config_as_message(task) + + # Assuming secret_names is a list of strings + image_pull_secret = [ + pb.ImagePullSecret(secret_name=secret_name) for secret_name in secret_names + ] + + msg.image_pull_secret.extend(image_pull_secret) + + task.platform_config['kubernetes'] = json_format.MessageToDict(msg) + + return task diff --git a/kubernetes_platform/python/kfp/kubernetes/pod_metadata.py b/kubernetes_platform/python/kfp/kubernetes/pod_metadata.py new file mode 100644 index 0000000000..dc8571b128 --- /dev/null +++ b/kubernetes_platform/python/kfp/kubernetes/pod_metadata.py @@ -0,0 +1,69 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from google.protobuf import json_format +from kfp.dsl import PipelineTask +from kfp.kubernetes import common + + +def add_pod_label( + task: PipelineTask, + label_key: str, + label_value: str, +) -> PipelineTask: + """Add a label to the task Pod's `metadata + `_. + + Each label is a key-value pair, corresponding to the metadata's `ObjectMeta PipelineTask: + """Add an annotation to the task Pod's `metadata + `_. + + Each annotation is a key-value pair, corresponding to the metadata's `ObjectMeta =4.21.1,<5', - # TODO: bump to 2.6.0 after next KFP SDK release with protobuf 4 - 'kfp>=2.5.0', + 'kfp>=2.6.0', ] DEV_REQUIREMENTS = [ 'docformatter==1.4', diff --git a/kubernetes_platform/python/test/snapshot/data/image_pull_secrets.py b/kubernetes_platform/python/test/snapshot/data/image_pull_secrets.py new file mode 100644 index 0000000000..5f5ed0f6d4 --- /dev/null +++ b/kubernetes_platform/python/test/snapshot/data/image_pull_secrets.py @@ -0,0 +1,32 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from kfp import dsl +from kfp import kubernetes + + +@dsl.component +def comp(): + pass + + +@dsl.pipeline +def my_pipeline(): + task = comp() + kubernetes.set_image_pull_secrets(task, ['my-secret']) + + +if __name__ == '__main__': + from kfp import compiler + compiler.Compiler().compile(my_pipeline, __file__.replace('.py', '.yaml')) diff --git a/kubernetes_platform/python/test/snapshot/data/image_pull_secrets.yaml b/kubernetes_platform/python/test/snapshot/data/image_pull_secrets.yaml new file mode 100644 index 0000000000..52c7f987a9 --- /dev/null +++ b/kubernetes_platform/python/test/snapshot/data/image_pull_secrets.yaml @@ -0,0 +1,57 @@ +# PIPELINE DEFINITION +# Name: my-pipeline +components: + comp-comp: + executorLabel: exec-comp +deploymentSpec: + executors: + exec-comp: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - comp + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef comp():\n pass\n\n" + image: python:3.7 +pipelineInfo: + name: my-pipeline +root: + dag: + tasks: + comp: + cachingOptions: + enableCache: true + componentRef: + name: comp-comp + taskInfo: + name: comp +schemaVersion: 2.1.0 +sdkVersion: kfp-2.6.0 +--- +platforms: + kubernetes: + deploymentSpec: + executors: + exec-comp: + imagePullSecret: + - secretName: my-secret diff --git a/kubernetes_platform/python/test/unit/test_image_pull_secrets.py b/kubernetes_platform/python/test/unit/test_image_pull_secrets.py new file mode 100644 index 0000000000..3aff349af8 --- /dev/null +++ b/kubernetes_platform/python/test/unit/test_image_pull_secrets.py @@ -0,0 +1,111 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from google.protobuf import json_format +from kfp import dsl +from kfp import kubernetes + + +class TestImagePullSecret: + + def test_add_one(self): + + @dsl.pipeline + def my_pipeline(): + task = comp() + kubernetes.set_image_pull_secrets(task, ['secret-name']) + + assert json_format.MessageToDict(my_pipeline.platform_spec) == { + 'platforms': { + 'kubernetes': { + 'deploymentSpec': { + 'executors': { + 'exec-comp': { + 'imagePullSecret': [{ + 'secretName': + 'secret-name' + }] + } + } + } + } + } + } + + def test_add_two(self): + + @dsl.pipeline + def my_pipeline(): + task = comp() + kubernetes.set_image_pull_secrets(task, ['secret-name1', 'secret-name2']) + + assert json_format.MessageToDict(my_pipeline.platform_spec) == { + 'platforms': { + 'kubernetes': { + 'deploymentSpec': { + 'executors': { + 'exec-comp': { + 'imagePullSecret': [{ + 'secretName': + 'secret-name1' + }, { + 'secretName': + 'secret-name2' + }, + ] + } + } + } + } + } + } + + def test_respects_other_configuration(self): + + @dsl.pipeline + def my_pipeline(): + task = comp() + + # Load the secret as a volume + kubernetes.use_secret_as_volume( + task, secret_name='secret-name', mount_path='/mnt/my_vol') + + # Set image pull secrets for a task using secret names + kubernetes.set_image_pull_secrets(task, ['secret-name']) + + assert json_format.MessageToDict(my_pipeline.platform_spec) == { + 'platforms': { + 'kubernetes': { + 'deploymentSpec': { + 'executors': { + 'exec-comp': { + 'secretAsVolume': [{ + 'secretName': 'secret-name', + 'mountPath': '/mnt/my_vol' + }], + 'imagePullSecret': [{ + 'secretName': + 'secret-name' + }] + } + } + } + } + } + } + + +@dsl.component +def comp(): + pass diff --git a/kubernetes_platform/python/test/unit/test_pod_metadata.py b/kubernetes_platform/python/test/unit/test_pod_metadata.py new file mode 100644 index 0000000000..1bb3c5ab5c --- /dev/null +++ b/kubernetes_platform/python/test/unit/test_pod_metadata.py @@ -0,0 +1,172 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from google.protobuf import json_format +from kfp import dsl +from kfp import kubernetes + + +class TestPodMetadata: + + def test_add_one(self): + + @dsl.pipeline + def my_pipeline(): + task = comp() + kubernetes.add_pod_label( + task, + label_key='kubeflow.com/kfp', + label_value='pipeline-node', + ) + + assert json_format.MessageToDict(my_pipeline.platform_spec) == { + 'platforms': { + 'kubernetes': { + 'deploymentSpec': { + 'executors': { + 'exec-comp': { + 'podMetadata': { + 'labels': { + 'kubeflow.com/kfp': 'pipeline-node' + } + } + } + } + } + } + } + } + + def test_add_same_one(self): + + @dsl.pipeline + def my_pipeline(): + task = comp() + kubernetes.add_pod_label( + task, + label_key='kubeflow.com/kfp', + label_value='pipeline-node', + ) + kubernetes.add_pod_label( + task, + label_key='kubeflow.com/kfp', + label_value='pipeline-node2', + ) + + assert json_format.MessageToDict(my_pipeline.platform_spec) == { + 'platforms': { + 'kubernetes': { + 'deploymentSpec': { + 'executors': { + 'exec-comp': { + 'podMetadata': { + 'labels': { + 'kubeflow.com/kfp': 'pipeline-node2' + } + } + } + } + } + } + } + } + + def test_add_two_and_mix(self): + + @dsl.pipeline + def my_pipeline(): + task = comp() + kubernetes.add_pod_label( + task, + label_key='kubeflow.com/kfp', + label_value='pipeline-node', + ) + kubernetes.add_pod_label( + task, + label_key='kubeflow.com/common', + label_value='test', + ) + kubernetes.add_pod_annotation( + task, + annotation_key='run_id', + annotation_value='123456', + ) + kubernetes.add_pod_annotation( + task, + annotation_key='experiment_id', + annotation_value='234567', + ) + + assert json_format.MessageToDict(my_pipeline.platform_spec) == { + 'platforms': { + 'kubernetes': { + 'deploymentSpec': { + 'executors': { + 'exec-comp': { + 'podMetadata': { + 'annotations': { + 'run_id': '123456', + 'experiment_id': '234567' + }, + 'labels': { + 'kubeflow.com/kfp': 'pipeline-node', + 'kubeflow.com/common': 'test' + } + } + } + } + } + } + } + } + + def test_respects_other_configuration(self): + + @dsl.pipeline + def my_pipeline(): + task = comp() + kubernetes.use_secret_as_volume( + task, secret_name='my-secret', mount_path='/mnt/my_vol') + kubernetes.add_pod_annotation( + task, + annotation_key='run_id', + annotation_value='123456', + ) + + assert json_format.MessageToDict(my_pipeline.platform_spec) == { + 'platforms': { + 'kubernetes': { + 'deploymentSpec': { + 'executors': { + 'exec-comp': { + 'podMetadata': { + 'annotations': { + 'run_id': '123456' + } + }, + 'secretAsVolume': [{ + 'secretName': 'my-secret', + 'mountPath': '/mnt/my_vol' + }] + } + } + } + } + } + } + + +@dsl.component +def comp(): + pass diff --git a/samples/test/metrics_visualization_v2.py b/samples/test/metrics_visualization_v2.py index fab3611013..f192b66729 100644 --- a/samples/test/metrics_visualization_v2.py +++ b/samples/test/metrics_visualization_v2.py @@ -89,6 +89,11 @@ def wine_classification(metrics: Output[ClassificationMetrics]): y_predict = cross_val_predict(rfc, X_train, y_train, cv=3, method='predict') fpr, tpr, thresholds = roc_curve( y_true=y_train, y_score=y_scores[:, 1], pos_label=True) + + # avoid inf thresholds + epsilon = 1e-6 + thresholds = [1 - epsilon if t == float('inf') else t for t in thresholds] + metrics.log_roc_curve(fpr, tpr, thresholds) diff --git a/sdk/RELEASE.md b/sdk/RELEASE.md index d408178691..c149f5b80c 100644 --- a/sdk/RELEASE.md +++ b/sdk/RELEASE.md @@ -10,6 +10,23 @@ ## Documentation updates +# 2.7.0 + +## Features +* Support local execution of sequential pipelines [\#10423](https://github.com/kubeflow/pipelines/pull/10423) +* Support local execution of `dsl.importer` components [\#10431](https://github.com/kubeflow/pipelines/pull/10431) +* Support local execution of pipelines in pipelines [\#10440](https://github.com/kubeflow/pipelines/pull/10440) +* Support `dsl.ParallelFor` over list of Artifacts [\#10441](https://github.com/kubeflow/pipelines/pull/10441) +* Fix bug where `dsl.OneOf` with multiple consumers cannot be compiled [\#10452](https://github.com/kubeflow/pipelines/pull/10452) + +## Breaking changes + +## Deprecations + +## Bug fixes and other changes +* Fix the compilation error when trying to iterate over a list of dictionaries with ParallelFor [\#10436](https://github.com/kubeflow/pipelines/pull/10436) +## Documentation updates + # 2.6.0 ## Features diff --git a/sdk/python/kfp/__init__.py b/sdk/python/kfp/__init__.py index c696ab3e5a..1193b8b27f 100644 --- a/sdk/python/kfp/__init__.py +++ b/sdk/python/kfp/__init__.py @@ -16,7 +16,7 @@ # https://packaging.python.org/guides/packaging-namespace-packages/#pkgutil-style-namespace-packages __path__ = __import__('pkgutil').extend_path(__path__, __name__) -__version__ = '2.6.0' +__version__ = '2.7.0' import sys import warnings diff --git a/sdk/python/kfp/cli/diagnose_me/utility_test.py b/sdk/python/kfp/cli/diagnose_me/utility_test.py index 0b3849372a..1bbf7895c5 100644 --- a/sdk/python/kfp/cli/diagnose_me/utility_test.py +++ b/sdk/python/kfp/cli/diagnose_me/utility_test.py @@ -25,31 +25,31 @@ class UtilityTest(unittest.TestCase): def test_execute_command_oserror(self): """Testing stdout and stderr is correctly captured upon OSError.""" response = utility.ExecutorResponse() - err_msg = "Testing handling of OSError" + err_msg = 'Testing handling of OSError' - with patch("subprocess.run") as mock_run: + with patch('subprocess.run') as mock_run: mock_run.side_effect = MagicMock(side_effect=OSError(err_msg)) response.execute_command([]) - self.assertEqual(response._stdout, "") + self.assertEqual(response._stdout, '') self.assertEqual(response._stderr, err_msg) def test_execute_command_stdout(self): """Testing stdout output is correctly captured.""" - test_string = "test string" + test_string = 'test string' response = utility.ExecutorResponse() - response.execute_command(["echo", test_string]) + response.execute_command(['echo', test_string]) - self.assertEqual(response._stdout, test_string + "\n") - self.assertEqual(response._stderr, "") + self.assertEqual(response._stdout, test_string + '\n') + self.assertEqual(response._stderr, '') def test_execute_command_stderr(self): """Testing stderr output is correctly captured.""" response = utility.ExecutorResponse() - response.execute_command(["ls", "not_a_real_dir"]) + response.execute_command(['ls', 'not_a_real_dir']) - self.assertEqual(response._stdout, "") - self.assertIn("No such file", response._stderr) + self.assertEqual(response._stdout, '') + self.assertIn('No such file', response._stderr) def test_parse_raw_input_json(self): """Testing json stdout is correctly parsed.""" diff --git a/sdk/python/kfp/compiler/compiler_test.py b/sdk/python/kfp/compiler/compiler_test.py index 2dbb86a7cb..8540842711 100644 --- a/sdk/python/kfp/compiler/compiler_test.py +++ b/sdk/python/kfp/compiler/compiler_test.py @@ -749,6 +749,41 @@ def my_pipeline(text: bool): pipeline_spec['root']['dag']['tasks']['for-loop-2'] ['iteratorPolicy']['parallelismLimit'], 2) + def test_compile_parallel_for_with_incompatible_input_type(self): + + @dsl.component + def producer_op(item: str) -> str: + return item + + @dsl.component + def list_dict_maker() -> List[Dict[str, int]]: + return [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}] + + with self.assertRaisesRegex( + type_utils.InconsistentTypeException, + "Incompatible argument passed to the input 'item' of component 'producer-op': Argument type 'NUMBER_INTEGER' is incompatible with the input type 'STRING'" + ): + + @dsl.pipeline + def my_pipeline(text: bool): + with dsl.ParallelFor(items=list_dict_maker().output) as item: + producer_task = producer_op(item=item.a) + + def test_compile_parallel_for_with_relaxed_type_checking(self): + + @dsl.component + def producer_op(item: str) -> str: + return item + + @dsl.component + def list_dict_maker() -> List[Dict]: + return [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}] + + @dsl.pipeline + def my_pipeline(text: bool): + with dsl.ParallelFor(items=list_dict_maker().output) as item: + producer_task = producer_op(item=item.a) + def test_compile_parallel_for_with_invalid_parallelism(self): @dsl.component @@ -4786,6 +4821,12 @@ def flip_coin_pipeline(execute_pipeline: bool): x = dsl.OneOf(print_task_1.outputs['a'], print_task_2.outputs['a']) print_artifact(a=x) + # test can be consumed multiple times from same oneof object + print_artifact(a=x) + y = dsl.OneOf(print_task_1.outputs['a'], + print_task_2.outputs['a']) + # test can be consumed multiple times from different equivalent oneof objects + print_artifact(a=y) # hole punched through if self.assertEqual( diff --git a/sdk/python/kfp/compiler/compiler_utils.py b/sdk/python/kfp/compiler/compiler_utils.py index 3a54cb041c..029b93c802 100644 --- a/sdk/python/kfp/compiler/compiler_utils.py +++ b/sdk/python/kfp/compiler/compiler_utils.py @@ -247,7 +247,8 @@ def get_inputs_for_all_groups( channel_to_add = channel while isinstance(channel_to_add, ( - for_loop.LoopArgument, + for_loop.LoopParameterArgument, + for_loop.LoopArtifactArgument, for_loop.LoopArgumentVariable, )): channels_to_add.append(channel_to_add) @@ -309,10 +310,11 @@ def get_inputs_for_all_groups( # loop items, we have to go from bottom-up because the # PipelineChannel can be originated from the middle a DAG, # which is not needed and visible to its parent DAG. - if isinstance( - channel, - (for_loop.LoopArgument, for_loop.LoopArgumentVariable - )) and channel.is_with_items_loop_argument: + if isinstance(channel, ( + for_loop.LoopParameterArgument, + for_loop.LoopArtifactArgument, + for_loop.LoopArgumentVariable, + )) and channel.is_with_items_loop_argument: for group_name in task_name_to_parent_groups[ task.name][::-1]: @@ -520,6 +522,15 @@ def get_outputs_for_all_groups( break elif isinstance(channel, pipeline_channel.OneOfMixin): + if channel in processed_oneofs: + continue + + # we want to mutate the oneof's inner channels ONLY where they + # are used in the oneof, not if they are used separately + # for example: we should only modify the copy of + # foo.output in dsl.OneOf(foo.output), not if foo.output is + # passed to another downstream task + channel.channels = [copy.copy(c) for c in channel.channels] for inner_channel in channel.channels: producer_task = pipeline.tasks[inner_channel.task_name] consumer_task = task @@ -546,9 +557,8 @@ def get_outputs_for_all_groups( outputs[upstream_name][channel.name] = channel break - # copy so we can update the inner channel for the next iteration - # use copy not deepcopy, since deepcopy will needlessly copy the entire pipeline - # this uses more memory than needed and some objects are uncopiable + # copy as a mechanism for "freezing" the inner channel + # before we make updates for the next iteration outputs[upstream_name][ surfaced_output_name] = copy.copy(inner_channel) @@ -594,6 +604,13 @@ def get_outputs_for_all_groups( # if the output has already been consumed by a task before it is returned, we don't need to reprocess it if channel in processed_oneofs: continue + + # we want to mutate the oneof's inner channels ONLY where they + # are used in the oneof, not if they are used separately + # for example: we should only modify the copy of + # foo.output in dsl.OneOf(foo.output), not if foo.output is passed + # to another downstream task + channel.channels = [copy.copy(c) for c in channel.channels] for inner_channel in channel.channels: producer_task = pipeline.tasks[inner_channel.task_name] upstream_groups = task_name_to_parent_groups[ @@ -613,9 +630,8 @@ def get_outputs_for_all_groups( outputs[upstream_name][channel.name] = channel break - # copy so we can update the inner channel for the next iteration - # use copy not deepcopy, since deepcopy will needlessly copy the entire pipeline - # this uses more memory than needed and some objects are uncopiable + # copy as a mechanism for "freezing" the inner channel + # before we make updates for the next iteration outputs[upstream_name][surfaced_output_name] = copy.copy( inner_channel) diff --git a/sdk/python/kfp/compiler/pipeline_spec_builder.py b/sdk/python/kfp/compiler/pipeline_spec_builder.py index a7e5546c10..86e446673e 100644 --- a/sdk/python/kfp/compiler/pipeline_spec_builder.py +++ b/sdk/python/kfp/compiler/pipeline_spec_builder.py @@ -23,6 +23,7 @@ from google.protobuf import json_format from google.protobuf import struct_pb2 import kfp +from kfp import dsl from kfp.compiler import compiler_utils from kfp.dsl import component_factory from kfp.dsl import for_loop @@ -128,8 +129,10 @@ def build_task_spec_for_task( task._task_spec.retry_policy.to_proto()) for input_name, input_value in task.inputs.items(): - # since LoopArgument and LoopArgumentVariable are narrower types than PipelineParameterChannel, start with it - if isinstance(input_value, for_loop.LoopArgument): + # Since LoopParameterArgument and LoopArtifactArgument and LoopArgumentVariable are narrower + # types than PipelineParameterChannel, start with them. + + if isinstance(input_value, for_loop.LoopParameterArgument): component_input_parameter = ( compiler_utils.additional_input_name_for_pipeline_channel( @@ -140,6 +143,17 @@ def build_task_spec_for_task( input_name].component_input_parameter = ( component_input_parameter) + elif isinstance(input_value, for_loop.LoopArtifactArgument): + + component_input_artifact = ( + compiler_utils.additional_input_name_for_pipeline_channel( + input_value)) + assert component_input_artifact in parent_component_inputs.artifacts, \ + f'component_input_artifact: {component_input_artifact} not found. All inputs: {parent_component_inputs}' + pipeline_task_spec.inputs.artifacts[ + input_name].component_input_artifact = ( + component_input_artifact) + elif isinstance(input_value, for_loop.LoopArgumentVariable): component_input_parameter = ( @@ -155,7 +169,7 @@ def build_task_spec_for_task( f'parseJson(string_value)["{input_value.subvar_name}"]') elif isinstance(input_value, pipeline_channel.PipelineArtifactChannel) or ( - isinstance(input_value, for_loop.Collected) and + isinstance(input_value, dsl.Collected) and input_value.is_artifact_channel): if input_value.task_name: @@ -190,7 +204,7 @@ def build_task_spec_for_task( elif isinstance(input_value, pipeline_channel.PipelineParameterChannel) or ( - isinstance(input_value, for_loop.Collected) and + isinstance(input_value, dsl.Collected) and not input_value.is_artifact_channel): if input_value.task_name: @@ -683,19 +697,25 @@ def build_component_spec_for_group( input_name = compiler_utils.additional_input_name_for_pipeline_channel( channel) - if isinstance(channel, pipeline_channel.PipelineArtifactChannel): + if isinstance(channel, (pipeline_channel.PipelineArtifactChannel, + for_loop.LoopArtifactArgument)): component_spec.input_definitions.artifacts[ input_name].artifact_type.CopyFrom( type_utils.bundled_artifact_to_artifact_proto( channel.channel_type)) component_spec.input_definitions.artifacts[ input_name].is_artifact_list = channel.is_artifact_list - else: - # channel is one of PipelineParameterChannel, LoopArgument, or - # LoopArgumentVariable. + elif isinstance(channel, + (pipeline_channel.PipelineParameterChannel, + for_loop.LoopParameterArgument, + for_loop.LoopArgumentVariable, dsl.Collected)): component_spec.input_definitions.parameters[ input_name].parameter_type = type_utils.get_parameter_type( channel.channel_type) + else: + raise TypeError( + f'Expected PipelineParameterChannel, PipelineArtifactChannel, LoopParameterArgument, LoopArtifactArgument, LoopArgumentVariable, or Collected, got {type(channel)}.' + ) for output_name, output in output_pipeline_channels.items(): if isinstance(output, pipeline_channel.PipelineArtifactChannel): @@ -747,13 +767,34 @@ def _update_task_spec_for_loop_group( loop_argument_item_name = compiler_utils.additional_input_name_for_pipeline_channel( group.loop_argument.full_name) - loop_arguments_item = f'{input_parameter_name}-{for_loop.LoopArgument.LOOP_ITEM_NAME_BASE}' + loop_arguments_item = f'{input_parameter_name}-{for_loop.LOOP_ITEM_NAME_BASE}' assert loop_arguments_item == loop_argument_item_name - pipeline_task_spec.parameter_iterator.items.input_parameter = ( - input_parameter_name) - pipeline_task_spec.parameter_iterator.item_input = ( - loop_argument_item_name) + if isinstance(group.loop_argument, for_loop.LoopParameterArgument): + pipeline_task_spec.parameter_iterator.items.input_parameter = ( + input_parameter_name) + pipeline_task_spec.parameter_iterator.item_input = ( + loop_argument_item_name) + + _pop_input_from_task_spec( + task_spec=pipeline_task_spec, + input_name=pipeline_task_spec.parameter_iterator.item_input) + + elif isinstance(group.loop_argument, for_loop.LoopArtifactArgument): + input_artifact_name = compiler_utils.additional_input_name_for_pipeline_channel( + loop_items_channel) + + pipeline_task_spec.artifact_iterator.items.input_artifact = input_artifact_name + pipeline_task_spec.artifact_iterator.item_input = ( + loop_argument_item_name) + + _pop_input_from_task_spec( + task_spec=pipeline_task_spec, + input_name=pipeline_task_spec.artifact_iterator.item_input) + else: + raise TypeError( + f'Expected LoopParameterArgument or LoopArtifactArgument, got {type(group.loop_argument)}.' + ) # If the loop items itself is a loop arguments variable, handle the # subvar name. @@ -777,14 +818,14 @@ def _update_task_spec_for_loop_group( pipeline_task_spec.parameter_iterator.item_input = ( input_parameter_name) + _pop_input_from_task_spec( + task_spec=pipeline_task_spec, + input_name=pipeline_task_spec.parameter_iterator.item_input) + if (group.parallelism_limit > 0): pipeline_task_spec.iterator_policy.parallelism_limit = ( group.parallelism_limit) - _pop_input_from_task_spec( - task_spec=pipeline_task_spec, - input_name=pipeline_task_spec.parameter_iterator.item_input) - def _binary_operations_to_cel_conjunctive( operations: List[pipeline_channel.ConditionOperation]) -> str: @@ -1290,10 +1331,11 @@ def build_spec_by_group( for channel in subgroup_input_channels: # Skip 'withItems' loop arguments if it's from an inner loop. - if isinstance( - channel, - (for_loop.LoopArgument, for_loop.LoopArgumentVariable - )) and channel.is_with_items_loop_argument: + if isinstance(channel, ( + for_loop.LoopParameterArgument, + for_loop.LoopArtifactArgument, + for_loop.LoopArgumentVariable, + )) and channel.is_with_items_loop_argument: withitems_loop_arg_found_in_self_or_upstream = False for group_name in group_name_to_parent_groups[ subgroup.name][::-1]: @@ -1782,7 +1824,7 @@ def _rename_component_refs( def validate_pipeline_outputs_dict( pipeline_outputs_dict: Dict[str, pipeline_channel.PipelineChannel]): for channel in pipeline_outputs_dict.values(): - if isinstance(channel, for_loop.Collected): + if isinstance(channel, dsl.Collected): # this validation doesn't apply to Collected continue diff --git a/sdk/python/kfp/dsl/for_loop.py b/sdk/python/kfp/dsl/for_loop.py index 14e0a42773..170bd30d45 100644 --- a/sdk/python/kfp/dsl/for_loop.py +++ b/sdk/python/kfp/dsl/for_loop.py @@ -20,6 +20,9 @@ ItemList = List[Union[int, float, str, Dict[str, Any]]] +LOOP_ITEM_NAME_BASE = 'loop-item' +LOOP_ITEM_PARAM_NAME_BASE = 'loop-item-param' + def _get_loop_item_type(type_name: str) -> Optional[str]: """Extracts the loop item type. @@ -64,24 +67,47 @@ def _get_subvar_type(type_name: str) -> Optional[str]: return match['value_type'].lstrip().rstrip() if match else None -class LoopArgument(pipeline_channel.PipelineParameterChannel): - """Represents the argument that are looped over in a ParallelFor loop. +def _get_first_element_type(item_list: ItemList) -> str: + """Returns the type of the first element of ItemList. + + Args: + item_list: List of items to loop over. If a list of dicts then, all dicts must have the same keys. + Returns: + A string representing the type of the first element (e.g., "int", "Dict[str, int]"). + """ + first_element = item_list[0] + if isinstance(first_element, dict): + key_type = type(list( + first_element.keys())[0]).__name__ # Get type of first key + value_type = type(list( + first_element.values())[0]).__name__ # Get type of first value + return f'Dict[{key_type}, {value_type}]' + else: + return type(first_element).__name__ + + +def _make_name(code: str) -> str: + """Makes a name for a loop argument from a unique code.""" + return f'{LOOP_ITEM_PARAM_NAME_BASE}-{code}' + + +class LoopParameterArgument(pipeline_channel.PipelineParameterChannel): + """Represents the parameter arguments that are looped over in a ParallelFor + loop. The class shouldn't be instantiated by the end user, rather it is created automatically by a ParallelFor ops group. - To create a LoopArgument instance, use one of its factory methods:: + To create a LoopParameterArgument instance, use one of its factory methods:: - LoopArgument.from_pipeline_channel(...) - LoopArgument.from_raw_items(...) + LoopParameterArgument.from_pipeline_channel(...) + LoopParameterArgument.from_raw_items(...) Attributes: items_or_pipeline_channel: The raw items or the PipelineParameterChannel object - this LoopArgument is associated to. + this LoopParameterArgument is associated to. """ - LOOP_ITEM_NAME_BASE = 'loop-item' - LOOP_ITEM_PARAM_NAME_BASE = 'loop-item-param' def __init__( self, @@ -90,7 +116,7 @@ def __init__( name_override: Optional[str] = None, **kwargs, ): - """Initializes a LoopArguments object. + """Initializes a LoopParameterArgument object. Args: items: List of items to loop over. If a list of dicts then, all @@ -108,7 +134,7 @@ def __init__( 'be specified.') if name_override is None: - super().__init__(name=self._make_name(name_code), **kwargs) + super().__init__(name=_make_name(name_code), **kwargs) else: super().__init__(name=name_override, **kwargs) @@ -149,20 +175,22 @@ def __getattr__(self, name: str): subvar_name=name, )) - def _make_name(self, code: str): - """Makes a name for this loop argument from a unique code.""" - return f'{self.LOOP_ITEM_PARAM_NAME_BASE}-{code}' - @classmethod def from_pipeline_channel( cls, channel: pipeline_channel.PipelineParameterChannel, - ) -> 'LoopArgument': - """Creates a LoopArgument object from a PipelineParameterChannel - object.""" - return LoopArgument( + ) -> 'LoopParameterArgument': + """Creates a LoopParameterArgument object from a + PipelineParameterChannel object. + + Provide a flexible default channel_type ('String') if extraction + from PipelineParameterChannel is unsuccessful. This maintains + compilation progress in cases of unknown or missing type + information. + """ + return LoopParameterArgument( items=channel, - name_override=channel.name + '-' + cls.LOOP_ITEM_NAME_BASE, + name_override=channel.name + '-' + LOOP_ITEM_NAME_BASE, task_name=channel.task_name, channel_type=_get_loop_item_type(channel.channel_type) or 'String', ) @@ -172,26 +200,89 @@ def from_raw_items( cls, raw_items: ItemList, name_code: str, - ) -> 'LoopArgument': - """Creates a LoopArgument object from raw item list.""" + ) -> 'LoopParameterArgument': + """Creates a LoopParameterArgument object from raw item list.""" if len(raw_items) == 0: raise ValueError('Got an empty item list for loop argument.') - return LoopArgument( + return LoopParameterArgument( items=raw_items, name_code=name_code, - channel_type=type(raw_items[0]).__name__, + channel_type=_get_first_element_type(raw_items), ) - @classmethod - def name_is_loop_argument(cls, name: str) -> bool: - """Returns True if the given channel name looks like a loop argument. - Either it came from a withItems loop item or withParams loop - item. +class LoopArtifactArgument(pipeline_channel.PipelineArtifactChannel): + """Represents the artifact arguments that are looped over in a ParallelFor + loop. + + The class shouldn't be instantiated by the end user, rather it is + created automatically by a ParallelFor ops group. + + To create a LoopArtifactArgument instance, use the factory method:: + + LoopArtifactArgument.from_pipeline_channel(...) + + + Attributes: + pipeline_channel: The PipelineArtifactChannel object this + LoopArtifactArgument is associated to. + """ + + def __init__( + self, + items: pipeline_channel.PipelineArtifactChannel, + name_code: Optional[str] = None, + name_override: Optional[str] = None, + **kwargs, + ): + """Initializes a LoopArtifactArgument object. + + Args: + items: The PipelineArtifactChannel object this LoopArtifactArgument is + associated to. + name_code: A unique code used to identify these loop arguments. + Should match the code for the ParallelFor ops_group which created + these LoopArtifactArguments. This prevents parameter name collisions. + name_override: The override name for PipelineArtifactChannel. + **kwargs: Any other keyword arguments passed down to PipelineArtifactChannel. """ - return ('-' + cls.LOOP_ITEM_NAME_BASE) in name \ - or (cls.LOOP_ITEM_PARAM_NAME_BASE + '-') in name + if (name_code is None) == (name_override is None): + raise ValueError( + 'Expect one and only one of `name_code` and `name_override` to ' + 'be specified.') + + # We don't support nested lists so `is_artifact_list` is always False. + if name_override is None: + super().__init__( + name=_make_name(name_code), is_artifact_list=False, **kwargs) + else: + super().__init__( + name=name_override, is_artifact_list=False, **kwargs) + + self.items_or_pipeline_channel = items + self.is_with_items_loop_argument = not isinstance( + items, pipeline_channel.PipelineArtifactChannel) + + @classmethod + def from_pipeline_channel( + cls, + channel: pipeline_channel.PipelineArtifactChannel, + ) -> 'LoopArtifactArgument': + """Creates a LoopArtifactArgument object from a PipelineArtifactChannel + object.""" + if not channel.is_artifact_list: + raise ValueError( + 'Cannot iterate over a single Artifact using `dsl.ParallelFor`. Expected a list of Artifacts as argument to `items`.' + ) + return LoopArtifactArgument( + items=channel, + name_override=channel.name + '-' + LOOP_ITEM_NAME_BASE, + task_name=channel.task_name, + channel_type=channel.channel_type, + ) + + # TODO: support artifact constants here. class LoopArgumentVariable(pipeline_channel.PipelineParameterChannel): @@ -215,13 +306,13 @@ class LoopArgumentVariable(pipeline_channel.PipelineParameterChannel): def __init__( self, - loop_argument: LoopArgument, + loop_argument: LoopParameterArgument, subvar_name: str, ): """Initializes a LoopArgumentVariable instance. Args: - loop_argument: The LoopArgument object this subvariable is based on + loop_argument: The LoopParameterArgument object this subvariable is based on a subvariable to. subvar_name: The name of this subvariable, which is the name of the dict key that spawned this subvariable. @@ -236,7 +327,7 @@ def __init__( self.subvar_name = subvar_name self.loop_argument = loop_argument - + # Handle potential channel_type extraction errors from LoopArgument by defaulting to 'String'. This maintains compilation progress. super().__init__( name=self._get_name_override( loop_arg_name=loop_argument.name, @@ -251,7 +342,7 @@ def __init__( def items_or_pipeline_channel( self) -> Union[ItemList, pipeline_channel.PipelineParameterChannel]: """Returns the loop argument items.""" - return self.loop_argument.items_or_pipeline_chanenl + return self.loop_argument.items_or_pipeline_channel @property def is_with_items_loop_argument(self) -> bool: @@ -301,16 +392,17 @@ def __init__( output: pipeline_channel.PipelineChannel, ) -> None: self.output = output + # we know all dsl.Collected instances are lists, so set `is_artifact_list` + # for type checking, which occurs before dsl.Collected is updated to + # it's "correct" channel during compilation if isinstance(output, pipeline_channel.PipelineArtifactChannel): channel_type = output.channel_type self.is_artifact_channel = True - # we know all dsl.Collected instances are lists, so set to true - # for type checking, which occurs before dsl.Collected is updated to - # it's "correct" channel during compilation self.is_artifact_list = True else: channel_type = 'LIST' self.is_artifact_channel = False + self.is_artifact_list = False super().__init__( output.name, diff --git a/sdk/python/kfp/dsl/for_loop_test.py b/sdk/python/kfp/dsl/for_loop_test.py index 7d1559c87b..266ad6c0de 100644 --- a/sdk/python/kfp/dsl/for_loop_test.py +++ b/sdk/python/kfp/dsl/for_loop_test.py @@ -19,6 +19,15 @@ from kfp.dsl import pipeline_channel +def name_is_loop_argument(name: str) -> bool: + """Returns True if the given channel name looks like a loop argument. + + Either it came from a withItems loop item or withParams loop item. + """ + return ('-' + for_loop.LOOP_ITEM_NAME_BASE) in name \ + or (for_loop.LOOP_ITEM_PARAM_NAME_BASE + '-') in name + + class ForLoopTest(parameterized.TestCase): @parameterized.parameters( @@ -68,6 +77,35 @@ def test_get_loop_item_type(self, collection_type, item_type): def test_get_subvar_type(self, dict_type, value_type): self.assertEqual(for_loop._get_subvar_type(dict_type), value_type) + @parameterized.parameters( + { + 'item_list': [ + { + 'A_a': 1 + }, + { + 'A_a': 2 + }, + ], + 'value_type': 'Dict[str, int]', + }, + { + 'item_list': [1, 2, 3], + 'value_type': 'int', + }, + { + 'item_list': ['a', 'b', 'c'], + 'value_type': 'str', + }, + { + 'item_list': [2.3, 4.5, 3.5], + 'value_type': 'float', + }, + ) + def test_get_first_element_type(self, item_list, value_type): + self.assertEqual( + for_loop._get_first_element_type(item_list), value_type) + @parameterized.parameters( { 'channel': @@ -88,13 +126,74 @@ def test_get_subvar_type(self, dict_type, value_type): 'expected_serialization_value': '{{channel:task=task1;name=output1-loop-item;type=Dict[str, str];}}', }, + { + 'channel': + pipeline_channel.PipelineParameterChannel( + name='output2', + channel_type='List[Dict]', + task_name='task1', + ), + 'expected_serialization_value': + '{{channel:task=task1;name=output2-loop-item;type=Dict;}}', + }, ) - def test_loop_argument_from_pipeline_channel(self, channel, - expected_serialization_value): - loop_argument = for_loop.LoopArgument.from_pipeline_channel(channel) + def test_loop_parameter_argument_from_pipeline_channel( + self, channel, expected_serialization_value): + loop_argument = for_loop.LoopParameterArgument.from_pipeline_channel( + channel) self.assertEqual(loop_argument.items_or_pipeline_channel, channel) self.assertEqual(str(loop_argument), expected_serialization_value) + @parameterized.parameters( + { + 'channel': + pipeline_channel.PipelineArtifactChannel( + name='param1', + channel_type='system.Artifact@0.0.1', + task_name='task1', + is_artifact_list=True, + ), + 'expected_serialization_value': + '{{channel:task=task1;name=param1-loop-item;type=system.Artifact@0.0.1;}}', + }, + { + 'channel': + pipeline_channel.PipelineArtifactChannel( + name='output1', + channel_type='system.Dataset@0.0.1', + task_name='task1', + is_artifact_list=True, + ), + 'expected_serialization_value': + '{{channel:task=task1;name=output1-loop-item;type=system.Dataset@0.0.1;}}', + }, + ) + def test_loop_artifact_argument_from_pipeline_channel( + self, channel, expected_serialization_value): + loop_argument = for_loop.LoopArtifactArgument.from_pipeline_channel( + channel) + self.assertEqual(loop_argument.items_or_pipeline_channel, channel), + self.assertEqual(str(loop_argument), expected_serialization_value) + + @parameterized.parameters( + { + 'channel': + pipeline_channel.PipelineArtifactChannel( + name='param1', + channel_type='system.Artifact@0.0.1', + task_name='task1', + is_artifact_list=False, + ), + },) + def test_loop_artifact_argument_from_single_pipeline_channel_raises_error( + self, channel): + with self.assertRaisesRegex( + ValueError, + r'Cannot iterate over a single Artifact using `dsl\.ParallelFor`\. Expected a list of Artifacts as argument to `items`\.' + ): + loop_argument = for_loop.LoopArtifactArgument.from_pipeline_channel( + channel) + @parameterized.parameters( { 'raw_items': ['a', 'b', 'c'], @@ -115,12 +214,12 @@ def test_loop_argument_from_pipeline_channel(self, channel, 'name_code': '2', 'expected_serialization_value': - '{{channel:task=;name=loop-item-param-2;type=dict;}}', + '{{channel:task=;name=loop-item-param-2;type=Dict[str, int];}}', }, ) def test_loop_argument_from_raw_items(self, raw_items, name_code, expected_serialization_value): - loop_argument = for_loop.LoopArgument.from_raw_items( + loop_argument = for_loop.LoopParameterArgument.from_raw_items( raw_items, name_code) self.assertEqual(loop_argument.items_or_pipeline_channel, raw_items) self.assertEqual(str(loop_argument), expected_serialization_value) @@ -148,8 +247,7 @@ def test_loop_argument_from_raw_items(self, raw_items, name_code, }, ) def test_name_is_loop_argument(self, name, expected_result): - self.assertEqual( - for_loop.LoopArgument.name_is_loop_argument(name), expected_result) + self.assertEqual(name_is_loop_argument(name), expected_result) @parameterized.parameters( { @@ -178,7 +276,7 @@ def test_name_is_loop_argument(self, name, expected_result): }, ) def test_create_loop_argument_varaible(self, subvar_name, valid): - loop_argument = for_loop.LoopArgument.from_pipeline_channel( + loop_argument = for_loop.LoopParameterArgument.from_pipeline_channel( pipeline_channel.PipelineParameterChannel( name='param1', channel_type='List[Dict[str, str]]', diff --git a/sdk/python/kfp/dsl/pipeline_channel.py b/sdk/python/kfp/dsl/pipeline_channel.py index 6adb52525c..4731030709 100644 --- a/sdk/python/kfp/dsl/pipeline_channel.py +++ b/sdk/python/kfp/dsl/pipeline_channel.py @@ -267,6 +267,7 @@ def __init__( channel_type: The type of the pipeline channel. task_name: Optional; the name of the task that produces the pipeline channel. + is_artifact_list: True if `channel_type` represents a list of the artifact type. Raises: ValueError: If name or task_name contains invalid characters. diff --git a/sdk/python/kfp/dsl/pipeline_task.py b/sdk/python/kfp/dsl/pipeline_task.py index b3d168330a..2e82d23378 100644 --- a/sdk/python/kfp/dsl/pipeline_task.py +++ b/sdk/python/kfp/dsl/pipeline_task.py @@ -28,6 +28,7 @@ from kfp.dsl import structures from kfp.dsl import utils from kfp.dsl.types import type_utils +from kfp.local import pipeline_orchestrator from kfp.pipeline_spec import pipeline_spec_pb2 _register_task_handler = lambda task: utils.maybe_rename_for_k8s( @@ -190,13 +191,20 @@ def _execute_locally(self, args: Dict[str, Any]) -> None: from kfp.local import task_dispatcher if self.pipeline_spec is not None: - raise NotImplementedError( - 'Local pipeline execution is not currently supported.') - - self._outputs = task_dispatcher.run_single_component( - pipeline_spec=self.component_spec.to_pipeline_spec(), - arguments=args, - ) + self._outputs = pipeline_orchestrator.run_local_pipeline( + pipeline_spec=self.pipeline_spec, + arguments=args, + ) + elif self.component_spec is not None: + self._outputs = task_dispatcher.run_single_task( + pipeline_spec=self.component_spec.to_pipeline_spec(), + arguments=args, + ) + else: + # user should never hit this + raise ValueError( + 'One of pipeline_spec or component_spec must not be None for local execution.' + ) self.state = TaskState.FINAL @property diff --git a/sdk/python/kfp/dsl/structures.py b/sdk/python/kfp/dsl/structures.py index 84db5aeb4a..440f9a3940 100644 --- a/sdk/python/kfp/dsl/structures.py +++ b/sdk/python/kfp/dsl/structures.py @@ -684,14 +684,14 @@ def from_v1_component_spec( schema_version = type_utils._GOOGLE_TYPES_VERSION elif isinstance(type_, str) and type_.lower( - ) in type_utils._ARTIFACT_CLASSES_MAPPING: - artifact_class = type_utils._ARTIFACT_CLASSES_MAPPING[ + ) in type_utils.ARTIFACT_CLASSES_MAPPING: + artifact_class = type_utils.ARTIFACT_CLASSES_MAPPING[ type_.lower()] schema_title = artifact_class.schema_title schema_version = artifact_class.schema_version elif type_ is None or isinstance(type_, dict) or type_.lower( - ) not in type_utils._ARTIFACT_CLASSES_MAPPING: + ) not in type_utils.ARTIFACT_CLASSES_MAPPING: schema_title = artifact_types.Artifact.schema_title schema_version = artifact_types.Artifact.schema_version @@ -734,14 +734,14 @@ def from_v1_component_spec( schema_version = type_utils._GOOGLE_TYPES_VERSION elif isinstance(type_, str) and type_.lower( - ) in type_utils._ARTIFACT_CLASSES_MAPPING: - artifact_class = type_utils._ARTIFACT_CLASSES_MAPPING[ + ) in type_utils.ARTIFACT_CLASSES_MAPPING: + artifact_class = type_utils.ARTIFACT_CLASSES_MAPPING[ type_.lower()] schema_title = artifact_class.schema_title schema_version = artifact_class.schema_version elif type_ is None or isinstance(type_, dict) or type_.lower( - ) not in type_utils._ARTIFACT_CLASSES_MAPPING: + ) not in type_utils.ARTIFACT_CLASSES_MAPPING: schema_title = artifact_types.Artifact.schema_title schema_version = artifact_types.Artifact.schema_version diff --git a/sdk/python/kfp/dsl/tasks_group.py b/sdk/python/kfp/dsl/tasks_group.py index 3f0f758bbd..c19fed788d 100644 --- a/sdk/python/kfp/dsl/tasks_group.py +++ b/sdk/python/kfp/dsl/tasks_group.py @@ -454,20 +454,27 @@ def __init__( is_root=False, ) - if isinstance(items, pipeline_channel.PipelineChannel): - self.loop_argument = for_loop.LoopArgument.from_pipeline_channel( + if isinstance(items, pipeline_channel.PipelineParameterChannel): + self.loop_argument = for_loop.LoopParameterArgument.from_pipeline_channel( + items) + self.items_is_pipeline_channel = True + elif isinstance(items, pipeline_channel.PipelineArtifactChannel): + self.loop_argument = for_loop.LoopArtifactArgument.from_pipeline_channel( items) self.items_is_pipeline_channel = True else: - self.loop_argument = for_loop.LoopArgument.from_raw_items( + self.loop_argument = for_loop.LoopParameterArgument.from_raw_items( raw_items=items, name_code=pipeline_context.Pipeline.get_default_pipeline() .get_next_group_id(), ) self.items_is_pipeline_channel = False + # TODO: support artifact constants here. self.parallelism_limit = parallelism - def __enter__(self) -> for_loop.LoopArgument: + def __enter__( + self + ) -> Union[for_loop.LoopParameterArgument, for_loop.LoopArtifactArgument]: super().__enter__() return self.loop_argument diff --git a/sdk/python/kfp/dsl/tasks_group_test.py b/sdk/python/kfp/dsl/tasks_group_test.py index 40c68ab372..92bcf50588 100644 --- a/sdk/python/kfp/dsl/tasks_group_test.py +++ b/sdk/python/kfp/dsl/tasks_group_test.py @@ -26,7 +26,7 @@ def test_basic(self): loop_items = ['pizza', 'hotdog', 'pasta'] with pipeline_context.Pipeline('pipeline') as p: with tasks_group.ParallelFor(items=loop_items) as parallel_for: - loop_argument = for_loop.LoopArgument.from_raw_items( + loop_argument = for_loop.LoopParameterArgument.from_raw_items( loop_items, '1') self.assertEqual(parallel_for.group_type, 'for-loop') self.assertEqual(parallel_for.parallelism, 0) @@ -37,7 +37,7 @@ def test_parallelfor_valid_parallelism(self): with pipeline_context.Pipeline('pipeline') as p: with tasks_group.ParallelFor( items=loop_items, parallelism=3) as parallel_for: - loop_argument = for_loop.LoopArgument.from_raw_items( + loop_argument = for_loop.LoopParameterArgument.from_raw_items( loop_items, '1') self.assertEqual(parallel_for.group_type, 'for-loop') self.assertEqual(parallel_for.parallelism, 3) @@ -48,7 +48,7 @@ def test_parallelfor_zero_parallelism(self): with pipeline_context.Pipeline('pipeline') as p: with tasks_group.ParallelFor( items=loop_items, parallelism=0) as parallel_for: - loop_argument = for_loop.LoopArgument.from_raw_items( + loop_argument = for_loop.LoopParameterArgument.from_raw_items( loop_items, '1') self.assertEqual(parallel_for.group_type, 'for-loop') self.assertEqual(parallel_for.parallelism, 0) diff --git a/sdk/python/kfp/dsl/types/artifact_types.py b/sdk/python/kfp/dsl/types/artifact_types.py index 271de58c29..e91fe8081e 100644 --- a/sdk/python/kfp/dsl/types/artifact_types.py +++ b/sdk/python/kfp/dsl/types/artifact_types.py @@ -21,6 +21,10 @@ _MINIO_LOCAL_MOUNT_PREFIX = '/minio/' _S3_LOCAL_MOUNT_PREFIX = '/s3/' +GCS_REMOTE_PREFIX = 'gs://' +MINIO_REMOTE_PREFIX = 'minio://' +S3_REMOTE_PREFIX = 's3://' + class Artifact: """Represents a generic machine learning artifact. @@ -83,12 +87,13 @@ def path(self, path: str) -> None: self._set_path(path) def _get_path(self) -> Optional[str]: - if self.uri.startswith('gs://'): - return _GCS_LOCAL_MOUNT_PREFIX + self.uri[len('gs://'):] - elif self.uri.startswith('minio://'): - return _MINIO_LOCAL_MOUNT_PREFIX + self.uri[len('minio://'):] - elif self.uri.startswith('s3://'): - return _S3_LOCAL_MOUNT_PREFIX + self.uri[len('s3://'):] + if self.uri.startswith(GCS_REMOTE_PREFIX): + return _GCS_LOCAL_MOUNT_PREFIX + self.uri[len(GCS_REMOTE_PREFIX):] + elif self.uri.startswith(MINIO_REMOTE_PREFIX): + return _MINIO_LOCAL_MOUNT_PREFIX + self.uri[len(MINIO_REMOTE_PREFIX + ):] + elif self.uri.startswith(S3_REMOTE_PREFIX): + return _S3_LOCAL_MOUNT_PREFIX + self.uri[len(S3_REMOTE_PREFIX):] # uri == path for local execution return self.uri @@ -98,11 +103,11 @@ def _set_path(self, path: str) -> None: def convert_local_path_to_remote_path(path: str) -> str: if path.startswith(_GCS_LOCAL_MOUNT_PREFIX): - return 'gs://' + path[len(_GCS_LOCAL_MOUNT_PREFIX):] + return GCS_REMOTE_PREFIX + path[len(_GCS_LOCAL_MOUNT_PREFIX):] elif path.startswith(_MINIO_LOCAL_MOUNT_PREFIX): - return 'minio://' + path[len(_MINIO_LOCAL_MOUNT_PREFIX):] + return MINIO_REMOTE_PREFIX + path[len(_MINIO_LOCAL_MOUNT_PREFIX):] elif path.startswith(_S3_LOCAL_MOUNT_PREFIX): - return 's3://' + path[len(_S3_LOCAL_MOUNT_PREFIX):] + return S3_REMOTE_PREFIX + path[len(_S3_LOCAL_MOUNT_PREFIX):] return path diff --git a/sdk/python/kfp/dsl/types/custom_artifact_types.py b/sdk/python/kfp/dsl/types/custom_artifact_types.py index beba576dc6..afe85783ae 100644 --- a/sdk/python/kfp/dsl/types/custom_artifact_types.py +++ b/sdk/python/kfp/dsl/types/custom_artifact_types.py @@ -44,7 +44,7 @@ def get_param_to_custom_artifact_class(func: Callable) -> Dict[str, type]: typing.NamedTuple returns. """ param_to_artifact_cls: Dict[str, type] = {} - kfp_artifact_classes = set(type_utils._ARTIFACT_CLASSES_MAPPING.values()) + kfp_artifact_classes = set(type_utils.ARTIFACT_CLASSES_MAPPING.values()) signature = inspect.signature(func) for name, param in signature.parameters.items(): diff --git a/sdk/python/kfp/dsl/types/type_annotations.py b/sdk/python/kfp/dsl/types/type_annotations.py index cd6adb89d8..aa5776b998 100644 --- a/sdk/python/kfp/dsl/types/type_annotations.py +++ b/sdk/python/kfp/dsl/types/type_annotations.py @@ -105,10 +105,10 @@ def construct_type_for_inputpath_or_outputpath( type_.schema_version) elif isinstance( type_, - str) and type_.lower() in type_utils._ARTIFACT_CLASSES_MAPPING: + str) and type_.lower() in type_utils.ARTIFACT_CLASSES_MAPPING: # v1 artifact backward compat, e.g. dsl.OutputPath('Dataset') return type_utils.create_bundled_artifact_type( - type_utils._ARTIFACT_CLASSES_MAPPING[type_.lower()].schema_title) + type_utils.ARTIFACT_CLASSES_MAPPING[type_.lower()].schema_title) elif type_utils.get_parameter_type(type_): return type_ else: diff --git a/sdk/python/kfp/dsl/types/type_utils.py b/sdk/python/kfp/dsl/types/type_utils.py index 666ee7247b..bc8e54f446 100644 --- a/sdk/python/kfp/dsl/types/type_utils.py +++ b/sdk/python/kfp/dsl/types/type_utils.py @@ -13,7 +13,6 @@ # limitations under the License. """Utilities for component I/O type mapping.""" -from distutils import util import inspect import json from typing import Any, Callable, Dict, Optional, Type, Union @@ -28,7 +27,7 @@ PARAMETER_TYPES = Union[str, int, float, bool, dict, list] # ComponentSpec I/O types to DSL ontology artifact classes mapping. -_ARTIFACT_CLASSES_MAPPING = { +ARTIFACT_CLASSES_MAPPING = { 'artifact': artifact_types.Artifact, 'model': artifact_types.Model, 'dataset': artifact_types.Dataset, @@ -71,9 +70,27 @@ } +# copied from distutils.util, which was removed in Python 3.12 +# https://github.com/pypa/distutils/blob/fb5c5704962cd3f40c69955437da9a88f4b28567/distutils/util.py#L340-L353 +def strtobool(val): + """Convert a string representation of truth to true (1) or false (0). + + True values are 'y', 'yes', 't', 'true', 'on', and '1'; false values + are 'n', 'no', 'f', 'false', 'off', and '0'. Raises ValueError if + 'val' is anything else. + """ + val = val.lower() + if val in ('y', 'yes', 't', 'true', 'on', '1'): + return 1 + elif val in ('n', 'no', 'f', 'false', 'off', '0'): + return 0 + else: + raise ValueError('invalid truth value %r' % (val,)) + + def bool_cast_fn(default: Union[str, bool]) -> bool: if isinstance(default, str): - default = util.strtobool(default) == 1 + default = strtobool(default) == 1 return default @@ -263,6 +280,15 @@ def verify_type_compatibility( expected_type = expected_spec.type given_type = _get_type_string_from_component_argument(given_value) + # avoid circular imports + from kfp.dsl import for_loop + + # Workaround for potential type-checking issues during ParallelFor compilation: When LoopArgument or LoopArgumentVariable are involved and the expected type is 'String', we temporarily relax type enforcement to avoid blocking compilation. This is necessary due to potential information loss during the compilation step. + if isinstance(given_value, + (for_loop.LoopParameterArgument, + for_loop.LoopArgumentVariable)) and given_type == 'String': + return True + given_is_param = is_parameter_type(str(given_type)) if given_is_param: given_type = get_parameter_type_name(given_type) diff --git a/sdk/python/kfp/dsl/types/type_utils_test.py b/sdk/python/kfp/dsl/types/type_utils_test.py index ea96033330..457d2ba0bd 100644 --- a/sdk/python/kfp/dsl/types/type_utils_test.py +++ b/sdk/python/kfp/dsl/types/type_utils_test.py @@ -23,6 +23,7 @@ from kfp import dsl from kfp.dsl import base_component from kfp.dsl import Dataset +from kfp.dsl import for_loop from kfp.dsl import Input from kfp.dsl import Output from kfp.dsl import pipeline_channel @@ -713,6 +714,31 @@ class TestTypeChecking(parameterized.TestCase): 'is_compatible': False, }, + { + 'argument_value': + for_loop.LoopArgumentVariable( + loop_argument=for_loop.LoopParameterArgument + .from_pipeline_channel( + pipeline_channel.create_pipeline_channel( + 'Output-loop-item', 'String', + 'list-dict-without-type-maker-5')), + subvar_name='a'), + 'parameter_input_spec': + structures.InputSpec('Integer'), + 'is_compatible': + True, + }, + { + 'argument_value': + for_loop.LoopParameterArgument.from_pipeline_channel( + pipeline_channel.create_pipeline_channel( + 'Output-loop-item', 'String', + 'list-dict-without-type-maker-5')), + 'parameter_input_spec': + structures.InputSpec('Integer'), + 'is_compatible': + True, + }, ) def test_verify_type_compatibility( self, diff --git a/sdk/python/kfp/local/__init__.py b/sdk/python/kfp/local/__init__.py index 5428cdca4d..6848df9400 100755 --- a/sdk/python/kfp/local/__init__.py +++ b/sdk/python/kfp/local/__init__.py @@ -11,7 +11,8 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -"""The KFP local runner.""" +"""The `kfp.local` module contains objects for running KFP components +locally.""" from kfp.local.config import DockerRunner from kfp.local.config import init diff --git a/sdk/python/kfp/local/config.py b/sdk/python/kfp/local/config.py index aba0488b5b..9ea01d1836 100755 --- a/sdk/python/kfp/local/config.py +++ b/sdk/python/kfp/local/config.py @@ -17,6 +17,8 @@ import os from typing import Union +from kfp import local + class LocalRunnerType(abc.ABC): """The ABC for user-facing Runner configurations. @@ -85,6 +87,13 @@ def __init__( self.pipeline_root = pipeline_root self.raise_on_error = raise_on_error + @classmethod + def validate(cls): + if cls.instance is None: + raise RuntimeError( + f"Local environment not initialized. Please run '{local.__name__}.{init.__name__}()' before executing tasks locally." + ) + def init( # annotate with subclasses, not parent class, for more helpful ref docs diff --git a/sdk/python/kfp/local/config_test.py b/sdk/python/kfp/local/config_test.py index 60943f0a44..3e1ebb26c5 100755 --- a/sdk/python/kfp/local/config_test.py +++ b/sdk/python/kfp/local/config_test.py @@ -59,6 +59,21 @@ def test_local_runner_config_is_singleton(self): local.SubprocessRunner(use_venv=False)) self.assertFalse(instance.raise_on_error, False) + def test_validate_success(self): + config.LocalExecutionConfig( + pipeline_root='other/local/root', + runner=local.SubprocessRunner(use_venv=False), + raise_on_error=False, + ) + config.LocalExecutionConfig.validate() + + def test_validate_fail(self): + with self.assertRaisesRegex( + RuntimeError, + f"Local environment not initialized. Please run 'kfp\.local\.init\(\)' before executing tasks locally\." + ): + config.LocalExecutionConfig.validate() + class TestInitCalls(unittest.TestCase): diff --git a/sdk/python/kfp/local/dag_orchestrator.py b/sdk/python/kfp/local/dag_orchestrator.py new file mode 100644 index 0000000000..229705c4e2 --- /dev/null +++ b/sdk/python/kfp/local/dag_orchestrator.py @@ -0,0 +1,347 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Code for locally executing a DAG within a pipeline.""" +import copy +from typing import Any, Dict, List, Tuple + +from kfp.local import config +from kfp.local import graph_utils +from kfp.local import importer_handler +from kfp.local import io +from kfp.local import status +from kfp.pipeline_spec import pipeline_spec_pb2 + +Outputs = Dict[str, Any] + + +def run_dag( + pipeline_resource_name: str, + dag_component_spec: pipeline_spec_pb2.ComponentSpec, + executors: Dict[str, + pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec], + components: Dict[str, pipeline_spec_pb2.ComponentSpec], + dag_arguments: Dict[str, Any], + pipeline_root: str, + runner: config.LocalRunnerType, + unique_pipeline_id: str, + fail_stack: List[str], +) -> Tuple[Outputs, status.Status]: + """Runs a DAGSpec. + + Args: + pipeline_resource_name: The root pipeline resource name. + dag_component_spec: The ComponentSpec which defines the DAG to execute. + executors: The ExecutorSpecs corresponding to the DAG. + components: The ComponentSpecs corresponding to the DAG. + dag_arguments: The arguments to the DAG's outer ComponentSpec. + io_store: The IOStore instance corresponding to this DAG. + pipeline_root: The local pipeline root. + runner: The user-specified local runner. + unique_pipeline_id: A unique identifier for the pipeline for placeholder resolution. + fail_stack: Mutable stack of failures. If a primitive task in the DAG fails, the task name is appended. If a multi-task DAG fails, the DAG name is appended. If the pipeline executes successfully, fail_stack will be empty throughout the full local execution call stack. + + Returns: + A two-tuple of (outputs, status). If status is FAILURE, outputs is an empty dictionary. + """ + from kfp.local import task_dispatcher + + dag_arguments_with_defaults = join_user_inputs_and_defaults( + dag_arguments=dag_arguments, + dag_inputs_spec=dag_component_spec.input_definitions, + ) + + # prepare IOStore for DAG + io_store = io.IOStore() + for k, v in dag_arguments_with_defaults.items(): + io_store.put_parent_input(k, v) + + # execute tasks in order + dag_spec = dag_component_spec.dag + sorted_tasks = graph_utils.topological_sort_tasks(dag_spec.tasks) + while sorted_tasks: + task_name = sorted_tasks.pop() + task_spec = dag_spec.tasks[task_name] + # TODO: support control flow features + validate_task_spec_not_loop_or_condition(task_spec=task_spec) + component_name = task_spec.component_ref.name + component_spec = components[component_name] + implementation = component_spec.WhichOneof('implementation') + if implementation == 'dag': + # unlikely to exceed default max recursion depth of 1000 + outputs, task_status = run_dag( + pipeline_resource_name=pipeline_resource_name, + dag_component_spec=component_spec, + components=components, + executors=executors, + dag_arguments=make_task_arguments( + task_spec.inputs, + io_store, + ), + pipeline_root=pipeline_root, + runner=runner, + unique_pipeline_id=unique_pipeline_id, + fail_stack=fail_stack, + ) + + elif implementation == 'executor_label': + executor_spec = executors[component_spec.executor_label] + task_arguments = make_task_arguments( + task_inputs_spec=dag_spec.tasks[task_name].inputs, + io_store=io_store, + ) + + if executor_spec.WhichOneof('spec') == 'importer': + outputs, task_status = importer_handler.run_importer( + pipeline_resource_name=pipeline_resource_name, + component_name=component_name, + component_spec=component_spec, + executor_spec=executor_spec, + arguments=task_arguments, + pipeline_root=pipeline_root, + unique_pipeline_id=unique_pipeline_id, + ) + elif executor_spec.WhichOneof('spec') == 'container': + outputs, task_status = task_dispatcher.run_single_task_implementation( + pipeline_resource_name=pipeline_resource_name, + component_name=component_name, + component_spec=component_spec, + executor_spec=executor_spec, + arguments=task_arguments, + pipeline_root=pipeline_root, + runner=runner, + # let the outer pipeline raise the error + raise_on_error=False, + # components may consume input artifacts when passed from upstream + # outputs or parent component inputs + block_input_artifact=False, + # provide the same unique job id for each task for + # consistent placeholder resolution + unique_pipeline_id=unique_pipeline_id, + ) + else: + raise ValueError( + "Got unknown spec in ExecutorSpec. Only 'dsl.component', 'dsl.container_component', and 'dsl.importer' are supported in local pipeline execution." + ) + else: + raise ValueError( + f'Got unknown component implementation: {implementation}') + + if task_status == status.Status.FAILURE: + fail_stack.append(task_name) + return {}, status.Status.FAILURE + + # update IO store on success + elif task_status == status.Status.SUCCESS: + for key, output in outputs.items(): + io_store.put_task_output( + task_name, + key, + output, + ) + else: + raise ValueError(f'Got unknown task status: {task_status.name}') + + dag_outputs = get_dag_outputs( + dag_outputs_spec=dag_component_spec.dag.outputs, + io_store=io_store, + ) + return dag_outputs, status.Status.SUCCESS + + +def join_user_inputs_and_defaults( + dag_arguments: Dict[str, Any], + dag_inputs_spec: pipeline_spec_pb2.ComponentInputsSpec, +) -> Dict[str, Any]: + """Collects user-provided arguments and default arguments (when no user- + provided argument) into a dictionary. Returns the dictionary. + + Args: + dag_arguments: The user-provided arguments to the DAG. + dag_inputs_spec: The ComponentInputSpec for the DAG. + + Returns: + The complete DAG inputs, with defaults included where the user-provided argument is missing. + """ + from kfp.local import executor_output_utils + + copied_dag_arguments = copy.deepcopy(dag_arguments) + + for input_name, input_spec in dag_inputs_spec.parameters.items(): + if input_name not in copied_dag_arguments: + copied_dag_arguments[ + input_name] = executor_output_utils.pb2_value_to_python( + input_spec.default_value) + return copied_dag_arguments + + +def make_task_arguments( + task_inputs_spec: pipeline_spec_pb2.TaskInputsSpec, + io_store: io.IOStore, +) -> Dict[str, Any]: + """Obtains a dictionary of arguments required to execute the task + corresponding to TaskInputsSpec. + + Args: + task_inputs_spec: The TaskInputsSpec for the task. + io_store: The IOStore of the current DAG. Used to obtain task arguments which come from upstream task outputs and parent component inputs. + + Returns: + The arguments for the task. + """ + from kfp.local import executor_output_utils + + task_arguments = {} + # handle parameters + for input_name, input_spec in task_inputs_spec.parameters.items(): + + # handle constants + if input_spec.HasField('runtime_value'): + # runtime_value's value should always be constant for the v2 compiler + if input_spec.runtime_value.WhichOneof('value') != 'constant': + raise ValueError('Expected constant.') + task_arguments[ + input_name] = executor_output_utils.pb2_value_to_python( + input_spec.runtime_value.constant) + + # handle upstream outputs + elif input_spec.HasField('task_output_parameter'): + task_arguments[input_name] = io_store.get_task_output( + input_spec.task_output_parameter.producer_task, + input_spec.task_output_parameter.output_parameter_key, + ) + + # handle parent pipeline input parameters + elif input_spec.HasField('component_input_parameter'): + task_arguments[input_name] = io_store.get_parent_input( + input_spec.component_input_parameter) + + # TODO: support dsl.ExitHandler + elif input_spec.HasField('task_final_status'): + raise NotImplementedError( + "'dsl.ExitHandler' is not yet support for local execution.") + + else: + raise ValueError(f'Missing input for parameter {input_name}.') + + # handle artifacts + for input_name, input_spec in task_inputs_spec.artifacts.items(): + if input_spec.HasField('task_output_artifact'): + task_arguments[input_name] = io_store.get_task_output( + input_spec.task_output_artifact.producer_task, + input_spec.task_output_artifact.output_artifact_key, + ) + elif input_spec.HasField('component_input_artifact'): + task_arguments[input_name] = io_store.get_parent_input( + input_spec.component_input_artifact) + else: + raise ValueError(f'Missing input for artifact {input_name}.') + + return task_arguments + + +def get_dag_output_parameters( + dag_outputs_spec: pipeline_spec_pb2.DagOutputsSpec, + io_store: io.IOStore, +) -> Dict[str, Any]: + """Gets the DAG output parameter values from a DagOutputsSpec and the DAG's + IOStore. + + Args: + dag_outputs_spec: DagOutputsSpec corresponding to the DAG. + io_store: IOStore corresponding to the DAG. + + Returns: + The DAG output parameters. + """ + outputs = {} + for root_output_key, parameter_selector_spec in dag_outputs_spec.parameters.items( + ): + kind = parameter_selector_spec.WhichOneof('kind') + if kind == 'value_from_parameter': + value_from_parameter = parameter_selector_spec.value_from_parameter + outputs[root_output_key] = io_store.get_task_output( + value_from_parameter.producer_subtask, + value_from_parameter.output_parameter_key, + ) + elif kind == 'value_from_oneof': + raise NotImplementedError( + "'dsl.OneOf' is not yet supported in local execution.") + else: + raise ValueError( + f"Got unknown 'parameter_selector_spec' kind: {kind}") + return outputs + + +def get_dag_output_artifacts( + dag_outputs_spec: pipeline_spec_pb2.DagOutputsSpec, + io_store: io.IOStore, +) -> Dict[str, Any]: + """Gets the DAG output artifact values from a DagOutputsSpec and the DAG's + IOStore. + + Args: + dag_outputs_spec: DagOutputsSpec corresponding to the DAG. + io_store: IOStore corresponding to the DAG. + + Returns: + The DAG output artifacts. + """ + outputs = {} + for root_output_key, artifact_selector_spec in dag_outputs_spec.artifacts.items( + ): + len_artifact_selectors = len(artifact_selector_spec.artifact_selectors) + if len_artifact_selectors != 1: + raise ValueError( + f'Expected 1 artifact in ArtifactSelectorSpec. Got: {len_artifact_selectors}' + ) + artifact_selector = artifact_selector_spec.artifact_selectors[0] + outputs[root_output_key] = io_store.get_task_output( + artifact_selector.producer_subtask, + artifact_selector.output_artifact_key, + ) + return outputs + + +def get_dag_outputs( + dag_outputs_spec: pipeline_spec_pb2.DagOutputsSpec, + io_store: io.IOStore, +) -> Dict[str, Any]: + """Gets the DAG output values from a DagOutputsSpec and the DAG's IOStore. + + Args: + dag_outputs_spec: DagOutputsSpec corresponding to the DAG. + io_store: IOStore corresponding to the DAG. + + Returns: + The DAG outputs. + """ + output_params = get_dag_output_parameters( + dag_outputs_spec=dag_outputs_spec, + io_store=io_store, + ) + output_artifacts = get_dag_output_artifacts( + dag_outputs_spec=dag_outputs_spec, + io_store=io_store, + ) + return {**output_params, **output_artifacts} + + +def validate_task_spec_not_loop_or_condition( + task_spec: pipeline_spec_pb2.PipelineTaskSpec) -> None: + if task_spec.trigger_policy.condition: + raise NotImplementedError( + "'dsl.Condition' is not supported by local pipeline execution.") + elif task_spec.WhichOneof('iterator'): + raise NotImplementedError( + "'dsl.ParallelFor' is not supported by local pipeline execution.") diff --git a/sdk/python/kfp/local/executor_input_utils.py b/sdk/python/kfp/local/executor_input_utils.py index 3ba5b6fdda..82eaa9d5b9 100644 --- a/sdk/python/kfp/local/executor_input_utils.py +++ b/sdk/python/kfp/local/executor_input_utils.py @@ -17,6 +17,8 @@ from typing import Any, Dict from google.protobuf import json_format +from google.protobuf import struct_pb2 +from kfp import dsl from kfp.compiler import pipeline_spec_builder from kfp.dsl import utils from kfp.pipeline_spec import pipeline_spec_pb2 @@ -28,20 +30,21 @@ def construct_executor_input( component_spec: pipeline_spec_pb2.ComponentSpec, arguments: Dict[str, Any], task_root: str, + block_input_artifact: bool, ) -> pipeline_spec_pb2.ExecutorInput: """Constructs the executor input message for a task execution.""" input_parameter_keys = list( component_spec.input_definitions.parameters.keys()) + # need to also add injected input parameters for f-string + input_parameter_keys += [ + k for k, v in arguments.items() if not isinstance(v, dsl.Artifact) + ] input_artifact_keys = list( component_spec.input_definitions.artifacts.keys()) - if input_artifact_keys: + if input_artifact_keys and block_input_artifact: raise ValueError( 'Input artifacts are not yet supported for local execution.') - output_parameter_keys = list( - component_spec.output_definitions.parameters.keys()) - output_artifact_specs_dict = component_spec.output_definitions.artifacts - inputs = pipeline_spec_pb2.ExecutorInput.Inputs( parameter_values={ param_name: @@ -50,9 +53,18 @@ def construct_executor_input( .parameters[param_name].default_value for param_name in input_parameter_keys }, - # input artifact constants are not supported yet - artifacts={}, + # input artifact constants are not supported yet, + # except when passed from an upstream output or parent component input + artifacts={ + artifact_name: + dsl_artifact_to_artifact_list(arguments[artifact_name]) + for artifact_name, _ in + component_spec.input_definitions.artifacts.items() + }, ) + + output_parameter_keys = list( + component_spec.output_definitions.parameters.keys()) outputs = pipeline_spec_pb2.ExecutorInput.Outputs( parameters={ param_name: pipeline_spec_pb2.ExecutorInput.OutputParameter( @@ -60,12 +72,12 @@ def construct_executor_input( for param_name in output_parameter_keys }, artifacts={ - artifact_name: make_artifact_list( + artifact_name: artifact_type_schema_to_artifact_list( name=artifact_name, artifact_type=artifact_spec.artifact_type, task_root=task_root, ) for artifact_name, artifact_spec in - output_artifact_specs_dict.items() + component_spec.output_definitions.artifacts.items() }, output_file=os.path.join(task_root, _EXECUTOR_OUTPUT_FILE), ) @@ -116,7 +128,7 @@ def construct_local_task_root( ) -def make_artifact_list( +def artifact_type_schema_to_artifact_list( name: str, artifact_type: pipeline_spec_pb2.ArtifactTypeSchema, task_root: str, @@ -128,7 +140,29 @@ def make_artifact_list( type=artifact_type, uri=os.path.join(task_root, name), # metadata always starts empty for output artifacts - metadata={}, + metadata=struct_pb2.Struct(), + ) + ]) + + +def dict_to_protobuf_struct(d: Dict[str, Any]) -> struct_pb2.Struct: + """Converts a Python dictionary to a prototobuf Struct.""" + protobuf_struct = struct_pb2.Struct() + protobuf_struct.update(d) + return protobuf_struct + + +def dsl_artifact_to_artifact_list( + artifact: dsl.Artifact) -> pipeline_spec_pb2.ArtifactList: + """Converts a single dsl.Aritfact to a protobuf ArtifactList.""" + return pipeline_spec_pb2.ArtifactList(artifacts=[ + pipeline_spec_pb2.RuntimeArtifact( + name=artifact.name, + type=pipeline_spec_pb2.ArtifactTypeSchema( + schema_title=artifact.schema_title, + schema_version=artifact.schema_version), + uri=artifact.uri, + metadata=dict_to_protobuf_struct(artifact.metadata), ) ]) diff --git a/sdk/python/kfp/local/executor_input_utils_test.py b/sdk/python/kfp/local/executor_input_utils_test.py index 29fe312619..a46c21801b 100644 --- a/sdk/python/kfp/local/executor_input_utils_test.py +++ b/sdk/python/kfp/local/executor_input_utils_test.py @@ -16,6 +16,7 @@ import unittest from google.protobuf import json_format +from kfp import dsl from kfp.local import executor_input_utils from kfp.local import testing_utilities from kfp.pipeline_spec import pipeline_spec_pb2 @@ -76,6 +77,7 @@ def test_no_inputs(self): component_spec=component_spec, arguments=arguments, task_root=task_root, + block_input_artifact=True, ) expected = pipeline_spec_pb2.ExecutorInput() json_format.ParseDict( @@ -129,6 +131,7 @@ def test_various_io_types(self): component_spec=component_spec, arguments=arguments, task_root=task_root, + block_input_artifact=True, ) expected = pipeline_spec_pb2.ExecutorInput() json_format.ParseDict( @@ -166,7 +169,7 @@ def test_various_io_types(self): }, expected) self.assertEqual(actual, expected) - def test_input_artifacts_not_yet_supported(self): + def test_block_input_artifact(self): component_spec = pipeline_spec_pb2.ComponentSpec() json_format.ParseDict( { @@ -191,8 +194,130 @@ def test_input_artifacts_not_yet_supported(self): component_spec=component_spec, arguments=arguments, task_root=task_root, + block_input_artifact=True, ) + def test_allow_input_artifact(self): + component_spec = pipeline_spec_pb2.ComponentSpec() + json_format.ParseDict( + { + 'inputDefinitions': { + 'artifacts': { + 'in_artifact': { + 'artifactType': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + } + } + } + }, + 'executorLabel': 'exec-comp' + }, component_spec) + task_root = '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/comp' + arguments = { + 'in_artifact': + dsl.Artifact( + name='artifact', + uri='/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/prev-comp/artifact', + metadata={'foo': 'bar'}) + } + actual = executor_input_utils.construct_executor_input( + component_spec=component_spec, + arguments=arguments, + task_root=task_root, + # this param says input artifacts should be permitted + block_input_artifact=False, + ) + expected = pipeline_spec_pb2.ExecutorInput() + json_format.ParseDict( + { + 'inputs': { + 'artifacts': { + 'in_artifact': { + 'artifacts': [{ + 'name': + 'artifact', + 'type': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'uri': + '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/prev-comp/artifact', + 'metadata': { + 'foo': 'bar' + } + }] + } + } + }, + 'outputs': { + 'outputFile': + '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/comp/executor_output.json' + } + }, expected) + self.assertEqual(actual, expected) + + def test_fstring_case(self): + component_spec = pipeline_spec_pb2.ComponentSpec() + json_format.ParseDict( + { + 'inputDefinitions': { + 'parameters': { + 'string': { + 'parameterType': 'STRING' + } + } + }, + 'outputDefinitions': { + 'parameters': { + 'Output': { + 'parameterType': 'STRING' + } + } + }, + 'executorLabel': 'exec-identity' + }, component_spec) + expected_executor_input = pipeline_spec_pb2.ExecutorInput() + json_format.ParseDict( + { + 'inputs': { + 'parameterValues': { + 'pipelinechannel--string': + 'baz', + 'string': + "bar-{{$.inputs.parameters['pipelinechannel--string']}}" + } + }, + 'outputs': { + 'parameters': { + 'Output': { + 'outputFile': + '/foo/bar/local_outputs/my-pipeline-2024-01-26-11-10-57XX-530768/identity/Output' + } + }, + 'outputFile': + '/foo/bar/local_outputs/my-pipeline-2024-01-26-11-10-57XX-530768/identity/executor_output.json' + } + }, expected_executor_input) + actual_executor_input = executor_input_utils.construct_executor_input( + component_spec=component_spec, + arguments={ + 'pipelinechannel--string': + 'baz', + # covers the case of an f-string, where the value of + # string includes an interpolation of + # pipelinechannel--string + 'string': + "bar-{{$.inputs.parameters['pipelinechannel--string']}}" + }, + task_root='/foo/bar/local_outputs/my-pipeline-2024-01-26-11-10-57XX-530768/identity', + block_input_artifact=True, + ) + self.assertEqual( + expected_executor_input, + actual_executor_input, + ) + class TestExecutorInputToDict(unittest.TestCase): diff --git a/sdk/python/kfp/local/graph_utils.py b/sdk/python/kfp/local/graph_utils.py new file mode 100644 index 0000000000..3441ceefce --- /dev/null +++ b/sdk/python/kfp/local/graph_utils.py @@ -0,0 +1,79 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Graph algorithms which are useful for working with PipelineSpec.""" + +from typing import Dict, List, Set + +from kfp.pipeline_spec import pipeline_spec_pb2 + + +def topological_sort_tasks( + tasks: Dict[str, pipeline_spec_pb2.PipelineTaskSpec]) -> List[str]: + """Given a dictionary of task name to PipelineTaskSpec, obtains a + topologically sorted stack of task names. + + Args: + tasks: The tasks in the pipeline. + + Returns: + A totally ordered stack of tasks. Tasks should be executed in the order they are popped off the right side of the stack. + """ + dependency_map = build_dependency_map(tasks) + return topological_sort(dependency_map) + + +def build_dependency_map( + tasks: Dict[str, + pipeline_spec_pb2.PipelineTaskSpec]) -> Dict[str, List[str]]: + """Builds a dictionary of task name to all upstream task names + (dependencies). This is a data structure simplification step, which allows + for a general topological_sort sort implementation. + + Args: + tasks: The tasks in the pipeline. + + Returns: + An dictionary of task name to all upstream tasks. The key task depends on all value tasks being executed first. + """ + return { + task_name: task_details.dependent_tasks + for task_name, task_details in tasks.items() + } + + +def topological_sort(dependency_map: Dict[str, List[str]]) -> List[str]: + """Topologically sorts a dictionary of task names to upstream tasks. + + Args: + dependency_map: A dictionary of tasks name to a list of upstream tasks. The key task depends on all value tasks being executed first. + + Returns: + A totally ordered stack of tasks. Tasks should be executed in the order they are popped off the right side of the stack. + """ + + def dfs(node: str) -> None: + visited.add(node) + for neighbor in dependency_map[node]: + if neighbor not in visited: + dfs(neighbor) + result.append(node) + + # sort lists to force deterministic result + dependency_map = {k: sorted(v) for k, v in dependency_map.items()} + visited: Set[str] = set() + result = [] + for node in dependency_map: + if node not in visited: + dfs(node) + return result[::-1] diff --git a/sdk/python/kfp/local/graph_utils_test.py b/sdk/python/kfp/local/graph_utils_test.py new file mode 100644 index 0000000000..838952833e --- /dev/null +++ b/sdk/python/kfp/local/graph_utils_test.py @@ -0,0 +1,127 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Tests for graph_utils.py.""" +from typing import Any, Dict +import unittest + +from google.protobuf import json_format +from kfp.local import graph_utils +from kfp.pipeline_spec import pipeline_spec_pb2 + + +class TestBuildDependencyMap(unittest.TestCase): + + def test_simple(self): + tasks = { + k: make_pipeline_task_spec(v) + for k, v in SIMPLE_TASK_TOPOLOGY.items() + } + actual = graph_utils.build_dependency_map(tasks) + expected = {'identity': [], 'identity-2': ['identity']} + self.assertEqual(actual, expected) + + def test_complex(self): + tasks = { + k: make_pipeline_task_spec(v) + for k, v in COMPLEX_TASK_TOPOLOGY.items() + } + actual = graph_utils.build_dependency_map(tasks) + expected = { + 'add': [], + 'add-2': ['multiply'], + 'divide': ['add-2'], + 'multiply': ['add'], + 'printer': ['add', 'divide', 'multiply'] + } + self.assertEqual(actual, expected) + + +class TestTopologicalSort(unittest.TestCase): + + def test_empty_graph(self): + self.assertEqual(graph_utils.topological_sort({}), []) + + def test_simple_linear_graph(self): + graph = {'A': ['B'], 'B': ['C'], 'C': []} + actual = graph_utils.topological_sort(graph) + expected = ['A', 'B', 'C'] + self.assertEqual(actual, expected) + + def test_separate_components(self): + graph = {'A': ['B'], 'B': [], 'C': ['D'], 'D': []} + actual = graph_utils.topological_sort(graph) + expected = ['C', 'D', 'A', 'B'] + self.assertEqual(actual, expected) + + def test_complex_graph(self): + graph = {'A': ['B', 'C'], 'B': ['D'], 'C': ['D'], 'D': []} + actual = graph_utils.topological_sort(graph) + expected = ['A', 'C', 'B', 'D'] + self.assertEqual(actual, expected) + + +class TestTopologicalSortTasks(unittest.TestCase): + + def test_simple(self): + tasks = { + k: make_pipeline_task_spec(v) + for k, v in SIMPLE_TASK_TOPOLOGY.items() + } + actual = graph_utils.topological_sort_tasks(tasks) + expected = ['identity-2', 'identity'] + self.assertEqual(actual, expected) + + def test_complex(self): + tasks = { + k: make_pipeline_task_spec(v) + for k, v in COMPLEX_TASK_TOPOLOGY.items() + } + actual = graph_utils.topological_sort_tasks(tasks) + expected = ['printer', 'divide', 'add-2', 'multiply', 'add'] + self.assertEqual(actual, expected) + + +SIMPLE_TASK_TOPOLOGY = { + 'identity': {}, + 'identity-2': { + 'dependentTasks': ['identity'], + } +} + +COMPLEX_TASK_TOPOLOGY = { + 'add': {}, + 'add-2': { + 'dependentTasks': ['multiply'], + }, + 'divide': { + 'dependentTasks': ['add-2'], + }, + 'multiply': { + 'dependentTasks': ['add'], + }, + 'printer': { + 'dependentTasks': ['add', 'divide', 'multiply'], + } +} + + +def make_pipeline_task_spec( + d: Dict[str, Any]) -> pipeline_spec_pb2.PipelineTaskSpec: + spec = pipeline_spec_pb2.PipelineTaskSpec() + json_format.ParseDict(d, spec) + return spec + + +if __name__ == '__main__': + unittest.main() diff --git a/sdk/python/kfp/local/importer_handler.py b/sdk/python/kfp/local/importer_handler.py new file mode 100644 index 0000000000..5cc523f7a3 --- /dev/null +++ b/sdk/python/kfp/local/importer_handler.py @@ -0,0 +1,142 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Code for running a dsl.importer locally.""" +import logging +from typing import Any, Dict, Tuple +import warnings + +from google.protobuf import json_format +from kfp import dsl +from kfp.dsl.types import artifact_types +from kfp.dsl.types import type_utils +from kfp.local import logging_utils +from kfp.local import placeholder_utils +from kfp.local import status +from kfp.pipeline_spec import pipeline_spec_pb2 + +Outputs = Dict[str, Any] + + +def run_importer( + pipeline_resource_name: str, + component_name: str, + component_spec: pipeline_spec_pb2.ComponentSpec, + executor_spec: pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec, + arguments: Dict[str, Any], + pipeline_root: str, + unique_pipeline_id: str, +) -> Tuple[Outputs, status.Status]: + """Runs an importer component and returns a two-tuple of (outputs, status). + + Args: + pipeline_resource_name: The root pipeline resource name. + component_name: The name of the component. + component_spec: The ComponentSpec of the importer. + executor_spec: The ExecutorSpec of the importer. + arguments: The arguments to the importer, as determined by the TaskInputsSpec for the importer. + pipeline_root: The local pipeline root directory of the current pipeline. + unique_pipeline_id: A unique identifier for the pipeline for placeholder resolution. + + Returns: + A two-tuple of the output dictionary ({"artifact": }) and the status. The outputs dictionary will be empty when status is failure. + """ + from kfp.local import executor_input_utils + + task_resource_name = executor_input_utils.get_local_task_resource_name( + component_name) + task_name_for_logs = logging_utils.format_task_name(task_resource_name) + with logging_utils.local_logger_context(): + logging.info(f'Executing task {task_name_for_logs}') + + task_root = executor_input_utils.construct_local_task_root( + pipeline_root=pipeline_root, + pipeline_resource_name=pipeline_resource_name, + task_resource_name=task_resource_name, + ) + executor_input = executor_input_utils.construct_executor_input( + component_spec=component_spec, + arguments=arguments, + task_root=task_root, + block_input_artifact=True, + ) + uri = get_importer_uri( + importer_spec=executor_spec.importer, + executor_input=executor_input, + ) + metadata = json_format.MessageToDict(executor_spec.importer.metadata) + executor_input_dict = executor_input_utils.executor_input_to_dict( + executor_input=executor_input, + component_spec=component_spec, + ) + metadata = placeholder_utils.recursively_resolve_json_dict_placeholders( + metadata, + executor_input_dict=executor_input_dict, + pipeline_resource_name=pipeline_resource_name, + task_resource_name=task_resource_name, + pipeline_root=pipeline_root, + pipeline_job_id=unique_pipeline_id, + pipeline_task_id=placeholder_utils.make_random_id(), + ) + ArtifactCls = get_artifact_class_from_schema_title( + executor_spec.importer.type_schema.schema_title) + outputs = { + 'artifact': ArtifactCls( + name='artifact', + uri=uri, + metadata=metadata, + ) + } + with logging_utils.local_logger_context(): + logging.info( + f'Task {task_name_for_logs} finished with status {logging_utils.format_status(status.Status.SUCCESS)}' + ) + output_string = [ + f'Task {task_name_for_logs} outputs:', + *logging_utils.make_log_lines_for_outputs(outputs), + ] + logging.info('\n'.join(output_string)) + logging_utils.print_horizontal_line() + + return outputs, status.Status.SUCCESS + + +def get_importer_uri( + importer_spec: pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec, + executor_input: pipeline_spec_pb2.ExecutorInput, +) -> str: + value_or_runtime_param = importer_spec.artifact_uri.WhichOneof('value') + if value_or_runtime_param == 'constant': + uri = importer_spec.artifact_uri.constant.string_value + elif value_or_runtime_param == 'runtime_parameter': + uri = executor_input.inputs.parameter_values['uri'].string_value + else: + raise ValueError( + f'Got unknown value of artifact_uri: {value_or_runtime_param}') + + if uri.startswith(artifact_types.GCS_REMOTE_PREFIX) or uri.startswith( + artifact_types.S3_REMOTE_PREFIX) or uri.startswith( + artifact_types.MINIO_REMOTE_PREFIX): + warnings.warn( + f"It looks like you're using the remote file '{uri}' in a 'dsl.importer'. Note that you will only be able to read and write to/from local files using 'artifact.path' in local executed pipelines." + ) + + return uri + + +def get_artifact_class_from_schema_title(schema_title: str) -> dsl.Artifact: + system_prefix = 'system.' + if schema_title.startswith(system_prefix): + return type_utils.ARTIFACT_CLASSES_MAPPING[schema_title.lstrip( + system_prefix).lower()] + return dsl.Artifact diff --git a/sdk/python/kfp/local/importer_handler_test.py b/sdk/python/kfp/local/importer_handler_test.py new file mode 100644 index 0000000000..56a4b4467b --- /dev/null +++ b/sdk/python/kfp/local/importer_handler_test.py @@ -0,0 +1,392 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Tests for importer_handler_test.py.""" +import unittest + +from google.protobuf import json_format +from kfp import dsl +from kfp.local import importer_handler +from kfp.local import status +from kfp.local import testing_utilities +from kfp.pipeline_spec import pipeline_spec_pb2 + + +class TestRunImporter(testing_utilities.LocalRunnerEnvironmentTestCase): + + def test_uri_from_upstream(self): + component_spec_dict = { + 'inputDefinitions': { + 'parameters': { + 'metadata': { + 'parameterType': 'STRING' + }, + 'uri': { + 'parameterType': 'STRING' + } + } + }, + 'outputDefinitions': { + 'artifacts': { + 'artifact': { + 'artifactType': { + 'schemaTitle': 'system.Dataset', + 'schemaVersion': '0.0.1' + } + } + } + }, + 'executorLabel': 'exec-importer' + } + component_spec = pipeline_spec_pb2.ComponentSpec() + json_format.ParseDict(component_spec_dict, component_spec) + + executor_spec_dict = { + 'importer': { + 'artifactUri': { + 'runtimeParameter': 'uri' + }, + 'typeSchema': { + 'schemaTitle': 'system.Dataset', + 'schemaVersion': '0.0.1' + }, + 'metadata': { + 'foo': "{{$.inputs.parameters['metadata']}}" + } + } + } + executor_spec = pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec( + ) + json_format.ParseDict(executor_spec_dict, executor_spec) + + outputs, task_status = importer_handler.run_importer( + pipeline_resource_name='my-pipeline-2024-01-24-15-16-30-586674', + component_name='comp-importer', + component_spec=component_spec, + executor_spec=executor_spec, + arguments={ + 'metadata': 'bar', + 'uri': '/fizz/buzz' + }, + pipeline_root='/foo/bar', + unique_pipeline_id='19024073', + ) + expected_artifact = dsl.Dataset( + name='artifact', + uri='/fizz/buzz', + metadata={'foo': 'bar'}, + ) + self.assertEqual(outputs['artifact'].schema_title, + expected_artifact.schema_title) + self.assertEqual(outputs['artifact'].name, expected_artifact.name) + self.assertEqual(outputs['artifact'].uri, expected_artifact.uri) + self.assertEqual(outputs['artifact'].metadata, + expected_artifact.metadata) + self.assertEqual(task_status, status.Status.SUCCESS) + + def test_uri_constant(self): + component_spec_dict = { + 'inputDefinitions': { + 'parameters': { + 'metadata': { + 'parameterType': 'STRING' + }, + 'uri': { + 'parameterType': 'STRING' + } + } + }, + 'outputDefinitions': { + 'artifacts': { + 'artifact': { + 'artifactType': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + } + } + } + }, + 'executorLabel': 'exec-importer' + } + component_spec = pipeline_spec_pb2.ComponentSpec() + json_format.ParseDict(component_spec_dict, component_spec) + + executor_spec_dict = { + 'importer': { + 'artifactUri': { + 'constant': 'gs://path' + }, + 'typeSchema': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'metadata': { + 'foo': [ + "{{$.inputs.parameters['metadata']}}", + "{{$.inputs.parameters['metadata']}}" + ] + } + } + } + executor_spec = pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec( + ) + json_format.ParseDict(executor_spec_dict, executor_spec) + + outputs, task_status = importer_handler.run_importer( + pipeline_resource_name='my-pipeline-2024-01-24-15-16-30-586674', + component_name='comp-importer', + component_spec=component_spec, + executor_spec=executor_spec, + arguments={ + 'metadata': 'text', + 'uri': 'gs://path' + }, + pipeline_root='/foo/bar', + unique_pipeline_id='19024073', + ) + expected_artifact = dsl.Artifact( + name='artifact', + uri='gs://path', + metadata={'foo': ['text', 'text']}, + ) + self.assertEqual(outputs['artifact'].schema_title, + expected_artifact.schema_title) + self.assertEqual(outputs['artifact'].name, expected_artifact.name) + self.assertEqual(outputs['artifact'].uri, expected_artifact.uri) + self.assertEqual(outputs['artifact'].metadata, + expected_artifact.metadata) + self.assertEqual(task_status, status.Status.SUCCESS) + + +class TestGetImporterUri(unittest.TestCase): + + def test_constant(self): + importer_spec_dict = { + 'artifactUri': { + 'constant': '/foo/path' + }, + 'typeSchema': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'metadata': { + 'foo': 'bar' + } + } + importer_spec = pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec( + ) + json_format.ParseDict(importer_spec_dict, importer_spec) + executor_input_dict = { + 'inputs': { + 'parameterValues': { + 'uri': '/foo/path' + } + }, + 'outputs': { + 'artifacts': { + 'artifact': { + 'artifacts': [{ + 'name': 'artifact', + 'type': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'uri': '/pipeline_root/task/artifact', + 'metadata': {} + }] + } + }, + 'outputFile': '/pipeline_root/task/executor_output.json' + } + } + executor_input = pipeline_spec_pb2.ExecutorInput() + json_format.ParseDict(executor_input_dict, executor_input) + uri = importer_handler.get_importer_uri( + importer_spec=importer_spec, + executor_input=executor_input, + ) + self.assertEqual(uri, '/foo/path') + + def test_runtime_parameter(self): + importer_spec_dict = { + 'artifactUri': { + 'runtimeParameter': 'uri' + }, + 'typeSchema': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'metadata': { + 'foo': 'bar' + } + } + importer_spec = pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec( + ) + json_format.ParseDict(importer_spec_dict, importer_spec) + executor_input_dict = { + 'inputs': { + 'parameterValues': { + 'uri': '/fizz/buzz' + } + }, + 'outputs': { + 'artifacts': { + 'artifact': { + 'artifacts': [{ + 'name': 'artifact', + 'type': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'uri': '/pipeline_root/foo/task/artifact', + 'metadata': {} + }] + } + }, + 'outputFile': '/pipeline_root/task/executor_output.json' + } + } + executor_input = pipeline_spec_pb2.ExecutorInput() + json_format.ParseDict(executor_input_dict, executor_input) + uri = importer_handler.get_importer_uri( + importer_spec=importer_spec, + executor_input=executor_input, + ) + self.assertEqual(uri, '/fizz/buzz') + + def test_constant_warns(self): + importer_spec_dict = { + 'artifactUri': { + 'constant': 'gs://foo/bar' + }, + 'typeSchema': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'metadata': { + 'foo': 'bar' + } + } + importer_spec = pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec( + ) + json_format.ParseDict(importer_spec_dict, importer_spec) + executor_input_dict = { + 'inputs': { + 'parameterValues': { + 'uri': 'gs://foo/bar' + } + }, + 'outputs': { + 'artifacts': { + 'artifact': { + 'artifacts': [{ + 'name': 'artifact', + 'type': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'uri': '/pipeline_root/foo/task/artifact', + 'metadata': {} + }] + } + }, + 'outputFile': '/pipeline_root/task/executor_output.json' + } + } + executor_input = pipeline_spec_pb2.ExecutorInput() + json_format.ParseDict(executor_input_dict, executor_input) + with self.assertWarnsRegex( + UserWarning, + r"It looks like you're using the remote file 'gs://foo/bar' in a 'dsl\.importer'\. Note that you will only be able to read and write to/from local files using 'artifact\.path' in local executed pipelines\." + ): + uri = importer_handler.get_importer_uri( + importer_spec=importer_spec, + executor_input=executor_input, + ) + self.assertEqual(uri, 'gs://foo/bar') + + def test_runtime_parameter_warns(self): + importer_spec_dict = { + 'artifactUri': { + 'runtimeParameter': 'uri' + }, + 'typeSchema': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'metadata': { + 'foo': 'bar' + } + } + importer_spec = pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec( + ) + json_format.ParseDict(importer_spec_dict, importer_spec) + executor_input_dict = { + 'inputs': { + 'parameterValues': { + 'uri': 's3://fizz/buzz' + } + }, + 'outputs': { + 'artifacts': { + 'artifact': { + 'artifacts': [{ + 'name': 'artifact', + 'type': { + 'schemaTitle': 'system.Artifact', + 'schemaVersion': '0.0.1' + }, + 'uri': '/pipeline_root/foo/task/artifact', + 'metadata': {} + }] + } + }, + 'outputFile': '/pipeline_root/task/executor_output.json' + } + } + executor_input = pipeline_spec_pb2.ExecutorInput() + json_format.ParseDict(executor_input_dict, executor_input) + with self.assertWarnsRegex( + UserWarning, + r"It looks like you're using the remote file 's3://fizz/buzz' in a 'dsl\.importer'\. Note that you will only be able to read and write to/from local files using 'artifact\.path' in local executed pipelines\." + ): + uri = importer_handler.get_importer_uri( + importer_spec=importer_spec, + executor_input=executor_input, + ) + self.assertEqual(uri, 's3://fizz/buzz') + + +class TestGetArtifactClassForSchemaTitle(unittest.TestCase): + + def test_artifact(self): + actual = importer_handler.get_artifact_class_from_schema_title( + 'system.Artifact') + expected = dsl.Artifact + self.assertEqual(actual, expected) + + def test_classification_metrics(self): + actual = importer_handler.get_artifact_class_from_schema_title( + 'system.ClassificationMetrics') + expected = dsl.ClassificationMetrics + self.assertEqual(actual, expected) + + def test_not_system_type(self): + actual = importer_handler.get_artifact_class_from_schema_title( + 'unknown.Type') + expected = dsl.Artifact + self.assertEqual(actual, expected) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdk/python/kfp/local/io.py b/sdk/python/kfp/local/io.py new file mode 100644 index 0000000000..7e3fd150b7 --- /dev/null +++ b/sdk/python/kfp/local/io.py @@ -0,0 +1,101 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Object for storing task outputs in-memory during local execution.""" + +import collections +from typing import Any, Dict + + +class IOStore: + + def __init__(self): + + self._task_output_data: Dict[str, + Dict[str, + Any]] = collections.defaultdict(dict) + self._parent_input_data: Dict[str, Any] = {} + + def put_parent_input( + self, + key: str, + value: Any, + ) -> None: + """Persist the value of a parent component (i.e., parent pipeline) + input. + + Args: + key: Parent component input name. + value: Value associated with key. + """ + self._parent_input_data[key] = value + + def get_parent_input( + self, + key: str, + ) -> None: + """Get the value of the parent component (i.e., parent pipeline) input + named key. + + Args: + key: Parent component input name. + + Returns: + The output value. + """ + if key in self._parent_input_data: + return self._parent_input_data[key] + raise ValueError(f"Parent pipeline input argument '{key}' not found.") + + def put_task_output( + self, + task_name: str, + key: str, + value: Any, + ) -> None: + """Persist the value of an upstream task output. + + Args: + task_name: Upstream task name. + key: Output name. + value: Value associated with key. + """ + self._task_output_data[task_name][key] = value + + def get_task_output( + self, + task_name: str, + key: str, + ) -> Any: + """Get the value of an upstream task output. + + Args: + task_name: Upstream task name. + key: Output name. + + Returns: + The output value. + """ + common_exception_string = f"Tried to get output '{key}' from task '{task_name}'" + if task_name in self._task_output_data: + outputs = self._task_output_data[task_name] + else: + raise ValueError( + f"{common_exception_string}, but task '{task_name}' not found.") + + if key in outputs: + return outputs[key] + else: + raise ValueError( + f"{common_exception_string}, but task '{task_name}' has no output named '{key}'." + ) diff --git a/sdk/python/kfp/local/io_test.py b/sdk/python/kfp/local/io_test.py new file mode 100644 index 0000000000..c1056cdf50 --- /dev/null +++ b/sdk/python/kfp/local/io_test.py @@ -0,0 +1,95 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Tests for io.py.""" + +import unittest + +from kfp import dsl +from kfp.local import io + + +class IOStoreTest(unittest.TestCase): + + def test_task_not_found(self): + store = io.IOStore() + with self.assertRaisesRegex( + ValueError, + r"Tried to get output 'foo' from task 'my-task', but task 'my-task' not found\." + ): + store.get_task_output('my-task', 'foo') + + def test_output_not_found(self): + store = io.IOStore() + store.put_task_output('my-task', 'bar', 'baz') + with self.assertRaisesRegex( + ValueError, + r"Tried to get output 'foo' from task 'my-task', but task 'my-task' has no output named 'foo'\." + ): + store.get_task_output('my-task', 'foo') + + def test_parent_input_not_found(self): + store = io.IOStore() + with self.assertRaisesRegex( + ValueError, r"Parent pipeline input argument 'foo' not found."): + store.get_parent_input('foo') + + def test_put_and_get_task_output(self): + store = io.IOStore() + store.put_task_output('my-task', 'foo', 'bar') + store.put_task_output('my-task', 'baz', 'bat') + self.assertEqual( + store.get_task_output('my-task', 'foo'), + 'bar', + ) + self.assertEqual( + store.get_task_output('my-task', 'baz'), + 'bat', + ) + # test getting doesn't remove by getting twice + self.assertEqual( + store.get_task_output('my-task', 'baz'), + 'bat', + ) + + def test_put_and_get_parent_input(self): + store = io.IOStore() + store.put_parent_input('foo', 'bar') + store.put_parent_input('baz', 'bat') + self.assertEqual( + store.get_parent_input('foo'), + 'bar', + ) + self.assertEqual( + store.get_parent_input('baz'), + 'bat', + ) + # test getting doesn't remove by getting twice + self.assertEqual( + store.get_parent_input('baz'), + 'bat', + ) + + def test_put_and_get_task_output_with_artifact(self): + artifact = dsl.Artifact( + name='foo', uri='/my/uri', metadata={'foo': 'bar'}) + store = io.IOStore() + store.put_task_output('my-task', 'foo', artifact) + self.assertEqual( + store.get_task_output('my-task', 'foo'), + artifact, + ) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdk/python/kfp/local/logging_utils.py b/sdk/python/kfp/local/logging_utils.py index 54f559604c..5f9a7bb4ad 100644 --- a/sdk/python/kfp/local/logging_utils.py +++ b/sdk/python/kfp/local/logging_utils.py @@ -16,16 +16,23 @@ import contextlib import datetime import logging +import shutil import sys from typing import Any, Dict, Generator, List from kfp import dsl +from kfp.local import status class Color: + # color for task name CYAN = '\033[96m' + # color for status success GREEN = '\033[92m' + # color for status failure RED = '\033[91m' + # color for pipeline name + MAGENTA = '\033[95m' RESET = '\033[0m' @@ -139,3 +146,25 @@ def make_log_lines_for_outputs(outputs: Dict[str, Any]) -> List[str]: output_lines.append(f'{key_chars}{value}') return output_lines + + +def print_horizontal_line() -> None: + columns, _ = shutil.get_terminal_size(fallback=(80, 24)) + print('-' * columns) + + +def format_task_name(task_name: str) -> str: + return color_text(f'{task_name!r}', Color.CYAN) + + +def format_status(task_status: status.Status) -> str: + if task_status == status.Status.SUCCESS: + return color_text(task_status.name, Color.GREEN) + elif task_status == status.Status.FAILURE: + return color_text(task_status.name, Color.RED) + else: + raise ValueError(f'Got unknown status: {task_status}') + + +def format_pipeline_name(pipeline_name: str) -> str: + return color_text(f'{pipeline_name!r}', Color.MAGENTA) diff --git a/sdk/python/kfp/local/logging_utils_test.py b/sdk/python/kfp/local/logging_utils_test.py index 53a03a8913..dd2cf2336b 100644 --- a/sdk/python/kfp/local/logging_utils_test.py +++ b/sdk/python/kfp/local/logging_utils_test.py @@ -19,6 +19,7 @@ from kfp import dsl from kfp.local import logging_utils +from kfp.local import status class TestIndentedPrint(unittest.TestCase): @@ -202,5 +203,39 @@ def test_mix_params_and_artifacts(self): self.assertListEqual(actual, expected) +class TestFormatStatus(unittest.TestCase): + + def test_success_status(self): + self.assertEqual( + logging_utils.format_status(status.Status.SUCCESS), + '\x1b[92mSUCCESS\x1b[0m') + + def test_failure_status(self): + self.assertEqual( + logging_utils.format_status(status.Status.FAILURE), + '\x1b[91mFAILURE\x1b[0m') + + def test_invalid_status(self): + with self.assertRaisesRegex(ValueError, + r'Got unknown status: INVALID_STATUS'): + logging_utils.format_status('INVALID_STATUS') + + +class TestFormatTaskName(unittest.TestCase): + + def test(self): + self.assertEqual( + logging_utils.format_task_name('my-task'), + '\x1b[96m\'my-task\'\x1b[0m') + + +class TestFormatPipelineName(unittest.TestCase): + + def test(self): + self.assertEqual( + logging_utils.format_pipeline_name('my-pipeline'), + '\033[95m\'my-pipeline\'\033[0m') + + if __name__ == '__main__': unittest.main() diff --git a/sdk/python/kfp/local/pipeline_orchestrator.py b/sdk/python/kfp/local/pipeline_orchestrator.py new file mode 100644 index 0000000000..117673688c --- /dev/null +++ b/sdk/python/kfp/local/pipeline_orchestrator.py @@ -0,0 +1,139 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Code for locally executing a compiled pipeline.""" +import logging +from typing import Any, Dict, List + +from kfp.local import config +from kfp.local import dag_orchestrator +from kfp.local import logging_utils +from kfp.local import placeholder_utils +from kfp.local import status +from kfp.local import utils +from kfp.pipeline_spec import pipeline_spec_pb2 + + +def run_local_pipeline( + pipeline_spec: pipeline_spec_pb2.PipelineSpec, + arguments: Dict[str, Any], +) -> Dict[str, Any]: + """kfp.local's entrypoint for running a local pipeline. + + Args: + pipeline_spec: PipelineSpec to run. + arguments: User-provided arguments. + + Returns: + The pipeline outputs. + """ + + # validate and access all global state in this function, not downstream + config.LocalExecutionConfig.validate() + return _run_local_pipeline_implementation( + pipeline_spec=pipeline_spec, + arguments=arguments, + raise_on_error=config.LocalExecutionConfig.instance.raise_on_error, + pipeline_root=config.LocalExecutionConfig.instance.pipeline_root, + runner=config.LocalExecutionConfig.instance.runner, + ) + + +def _run_local_pipeline_implementation( + pipeline_spec: pipeline_spec_pb2.PipelineSpec, + arguments: Dict[str, Any], + raise_on_error: bool, + pipeline_root: str, + runner: config.LocalRunnerType, +) -> Dict[str, Any]: + """Implementation of run local pipeline. + + Args: + pipeline_spec: PipelineSpec to run. + arguments: User-provided arguments. + raise_on_error: Whether to raise an exception if a task exits with failure. + pipeline_root: The local pipeline root. + runner: The user-specified local runner. + + Returns: + The pipeline outputs. + """ + from kfp.local import executor_input_utils + + pipeline_name = pipeline_spec.pipeline_info.name + pipeline_resource_name = executor_input_utils.get_local_pipeline_resource_name( + pipeline_name) + pipeline_name_with_color = logging_utils.format_pipeline_name(pipeline_name) + + with logging_utils.local_logger_context(): + logging.info(f'Running pipeline: {pipeline_name_with_color}') + logging_utils.print_horizontal_line() + + executors = { + name: utils.struct_to_executor_spec(executor) for name, executor in + pipeline_spec.deployment_spec['executors'].items() + } + # convert to dict for consistency with executors + components = dict(pipeline_spec.components.items()) + fail_stack: List[str] = [] + outputs, dag_status = dag_orchestrator.run_dag( + pipeline_resource_name=pipeline_resource_name, + dag_component_spec=pipeline_spec.root, + executors=executors, + components=components, + dag_arguments=arguments, + pipeline_root=pipeline_root, + runner=runner, + unique_pipeline_id=placeholder_utils.make_random_id(), + fail_stack=fail_stack, + ) + if dag_status == status.Status.SUCCESS: + status_with_color = logging_utils.format_status(status.Status.SUCCESS) + with logging_utils.local_logger_context(): + logging.info( + f'Pipeline {pipeline_name_with_color} finished with status {status_with_color}' + ) + return outputs + elif dag_status == status.Status.FAILURE: + log_and_maybe_raise_for_failure( + pipeline_name=pipeline_name, + fail_stack=fail_stack, + raise_on_error=raise_on_error, + ) + return {} + else: + raise ValueError(f'Got unknown task status {dag_status.name}') + + +def log_and_maybe_raise_for_failure( + pipeline_name: str, + raise_on_error: bool, + fail_stack: List[str], +) -> None: + """To be called if an inner pipeline task exits with failure status. Either + logs error or throws exception, depending on raise_on_error. + + Args: + pipeline_name: The name of the root pipeline. + raise_on_error: Whether to raise on error. + fail_stack: The stack of task failures, if any, starting with the innermost task that failed to the outermost pipeline. Excludes the root pipeline. + """ + status_with_color = logging_utils.format_status(status.Status.FAILURE) + pipeline_name_with_color = logging_utils.format_pipeline_name(pipeline_name) + task_chain_with_color = ' inside '.join( + logging_utils.format_task_name(task_name) for task_name in fail_stack) + msg = f'Pipeline {pipeline_name_with_color} finished with status {status_with_color}. Inner task failed: {task_chain_with_color}.' + if raise_on_error: + raise RuntimeError(msg) + with logging_utils.local_logger_context(): + logging.error(msg) diff --git a/sdk/python/kfp/local/pipeline_orchestrator_test.py b/sdk/python/kfp/local/pipeline_orchestrator_test.py new file mode 100644 index 0000000000..49bf6829f9 --- /dev/null +++ b/sdk/python/kfp/local/pipeline_orchestrator_test.py @@ -0,0 +1,714 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Tests for pipeline_orchestrator.py.""" + +import io as stdlib_io +import os +from typing import NamedTuple +import unittest +from unittest import mock + +from kfp import dsl +from kfp import local +from kfp.dsl import Dataset +from kfp.dsl import Input +from kfp.dsl import Model +from kfp.dsl import Output +from kfp.dsl import pipeline_task +from kfp.local import testing_utilities + +ROOT_FOR_TESTING = './testing_root' + + +class TestRunLocalPipeline(testing_utilities.LocalRunnerEnvironmentTestCase): + + def assert_output_dir_contents( + self, + expected_dirs_in_pipeline_root: int, + expected_files_in_pipeline_dir: int, + ) -> None: + # check that output files are correctly nested + # and only one directory for the outer pipeline in pipeline root + actual_dirs_in_pipeline_root = os.listdir(ROOT_FOR_TESTING) + self.assertLen( + actual_dirs_in_pipeline_root, + expected_dirs_in_pipeline_root, + ) + + # and check that each task has a directory + actual_contents_of_pipeline_dir = os.listdir( + os.path.join( + ROOT_FOR_TESTING, + actual_dirs_in_pipeline_root[0], + )) + self.assertLen( + actual_contents_of_pipeline_dir, + expected_files_in_pipeline_dir, + ) + + def test_must_initialize(self): + + @dsl.component + def identity(string: str) -> str: + return string + + @dsl.pipeline + def my_pipeline(): + identity(string='foo') + + with self.assertRaisesRegex( + RuntimeError, + r"Local environment not initialized\. Please run 'kfp\.local\.init\(\)' before executing tasks locally\." + ): + my_pipeline() + + def test_no_io(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def pass_op(): + pass + + @dsl.pipeline + def my_pipeline(): + pass_op() + pass_op() + + result = my_pipeline() + self.assertIsInstance(result, pipeline_task.PipelineTask) + self.assertEqual(result.outputs, {}) + self.assert_output_dir_contents(1, 2) + + def test_missing_args(self): + local.init(local.SubprocessRunner()) + + @dsl.component + def identity(string: str) -> str: + return string + + @dsl.pipeline + def my_pipeline(string: str) -> str: + t1 = identity(string=string) + t2 = identity(string=t1.output) + return t2.output + + with self.assertRaisesRegex( + TypeError, + r'my-pipeline\(\) missing 1 required argument: string\.'): + my_pipeline() + + def test_single_return(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def identity(string: str) -> str: + return string + + @dsl.pipeline + def my_pipeline(string: str = 'text') -> str: + t1 = identity(string=string) + t2 = identity(string=t1.output) + return t2.output + + task = my_pipeline() + self.assertEqual(task.output, 'text') + self.assert_output_dir_contents(1, 2) + + def test_can_run_loaded_pipeline(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def identity(string: str) -> str: + return string + + @dsl.pipeline + def my_pipeline(string: str = 'text') -> str: + t1 = identity(string=string) + t2 = identity(string=t1.output) + return t2.output + + my_pipeline_loaded = testing_utilities.compile_and_load_component( + my_pipeline) + + task = my_pipeline_loaded(string='foo') + self.assertEqual(task.output, 'foo') + self.assert_output_dir_contents(1, 2) + + def test_all_param_io(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + # tests all I/O types with: + # - use of component args/defaults + # - use of pipeline args/defaults + # - passing pipeline args to first run component and not first run component + # - passing args from pipeline param, upstream output, and constant + # - pipeline surfacing outputs from last run component and not last run component + + @dsl.component + def many_parameter_component( + a_float: float, + a_boolean: bool, + a_dict: dict, + a_string: str = 'default', + an_integer: int = 12, + a_list: list = ['item1', 'item2'], + ) -> NamedTuple( + 'outputs', + a_string=str, + a_float=float, + an_integer=int, + a_boolean=bool, + a_list=list, + a_dict=dict, + ): + outputs = NamedTuple( + 'outputs', + a_string=str, + a_float=float, + an_integer=int, + a_boolean=bool, + a_list=list, + a_dict=dict, + ) + return outputs( + a_string=a_string, + a_float=a_float, + an_integer=an_integer, + a_boolean=a_boolean, + a_list=a_list, + a_dict=a_dict, + ) + + @dsl.pipeline + def my_pipeline( + flt: float, + boolean: bool, + dictionary: dict = {'foo': 'bar'}, + ) -> NamedTuple( + 'outputs', + another_string=str, + another_float=float, + another_integer=int, + another_boolean=bool, + another_list=list, + another_dict=dict, + ): + + t1 = many_parameter_component( + a_float=flt, + a_boolean=True, + a_dict={'baz': 'bat'}, + an_integer=10, + ) + t2 = many_parameter_component( + a_float=t1.outputs['a_float'], + a_dict=dictionary, + a_boolean=boolean, + ) + + outputs = NamedTuple( + 'outputs', + another_string=str, + another_float=float, + another_integer=int, + another_boolean=bool, + another_list=list, + another_dict=dict, + ) + return outputs( + another_string=t1.outputs['a_string'], + another_float=t1.outputs['a_float'], + another_integer=t1.outputs['an_integer'], + another_boolean=t2.outputs['a_boolean'], + another_list=t2.outputs['a_list'], + another_dict=t1.outputs['a_dict'], + ) + + task = my_pipeline( + flt=2.718, + boolean=False, + ) + self.assertEqual(task.outputs['another_string'], 'default') + self.assertEqual(task.outputs['another_float'], 2.718) + self.assertEqual(task.outputs['another_integer'], 10) + self.assertEqual(task.outputs['another_boolean'], False) + self.assertEqual(task.outputs['another_list'], ['item1', 'item2']) + self.assertEqual(task.outputs['another_dict'], {'baz': 'bat'}) + self.assert_output_dir_contents(1, 2) + + def test_artifact_io(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def make_dataset(content: str) -> Dataset: + d = Dataset(uri=dsl.get_uri(), metadata={'framework': 'pandas'}) + with open(d.path, 'w') as f: + f.write(content) + return d + + @dsl.component + def make_model(dataset: Input[Dataset], model: Output[Model]): + with open(dataset.path) as f: + content = f.read() + with open(model.path, 'w') as f: + f.write(content * 2) + model.metadata['framework'] = 'tensorflow' + model.metadata['dataset'] = dataset.metadata + + @dsl.pipeline + def my_pipeline(content: str = 'string') -> Model: + t1 = make_dataset(content=content) + t2 = make_model(dataset=t1.output) + return t2.outputs['model'] + + task = my_pipeline(content='text') + output_model = task.output + self.assertIsInstance(output_model, Model) + self.assertEqual(output_model.name, 'model') + self.assertTrue(output_model.uri.endswith('/make-model/model')) + self.assertEqual(output_model.metadata, { + 'framework': 'tensorflow', + 'dataset': { + 'framework': 'pandas' + } + }) + self.assert_output_dir_contents(1, 2) + + def test_input_artifact_constant_not_permitted(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def print_model(model: Input[Model]): + print(model.name) + print(model.uri) + print(model.metadata) + + with self.assertRaisesRegex( + ValueError, + r"Input artifacts are not supported\. Got input artifact of type 'Model'\." + ): + + @dsl.pipeline + def my_pipeline(): + print_model(model=dsl.Model(name='model', uri='/foo/bar/model')) + + def test_importer(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def artifact_printer(a: Dataset): + print(a) + + @dsl.component + def identity(string: str) -> str: + return string + + @dsl.pipeline + def my_pipeline(greeting: str) -> Dataset: + world_op = identity(string='world') + message_op = identity(string='message') + imp_op = dsl.importer( + artifact_uri='/local/path/to/dataset', + artifact_class=Dataset, + metadata={ + message_op.output: [greeting, world_op.output], + }) + artifact_printer(a=imp_op.outputs['artifact']) + return imp_op.outputs['artifact'] + + task = my_pipeline(greeting='hello') + output_model = task.output + self.assertIsInstance(output_model, Dataset) + self.assertEqual(output_model.name, 'artifact') + self.assertEqual(output_model.uri, '/local/path/to/dataset') + self.assertEqual(output_model.metadata, { + 'message': ['hello', 'world'], + }) + # importer doesn't have an output directory + self.assert_output_dir_contents(1, 3) + + def test_pipeline_in_pipeline_simple(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def identity(string: str) -> str: + return string + + @dsl.pipeline + def inner_pipeline() -> str: + return identity(string='foo').output + + @dsl.pipeline + def my_pipeline() -> str: + return inner_pipeline().output + + task = my_pipeline() + self.assertEqual(task.output, 'foo') + self.assert_output_dir_contents(1, 1) + + def test_pipeline_in_pipeline_complex(self): + local.init(local.SubprocessRunner(), pipeline_root=ROOT_FOR_TESTING) + + @dsl.component + def square(x: float) -> float: + return x**2 + + @dsl.component + def add(x: float, y: float) -> float: + return x + y + + @dsl.component + def square_root(x: float) -> float: + return x**.5 + + @dsl.component + def convert_to_artifact(val: float) -> Dataset: + dataset = Dataset(name='dataset', uri=dsl.get_uri()) + with open(dataset.path, 'w') as f: + f.write(str(val)) + + @dsl.component + def convert_from_artifact(dataset: Dataset) -> float: + with open(dataset.path) as f: + return float(f.read()) + + @dsl.pipeline + def square_and_sum(a: float, b: float) -> Dataset: + a_sq_task = square(x=a) + b_sq_task = square(x=b) + add_task = add(x=a_sq_task.output, y=b_sq_task.output) + return convert_to_artifact(val=add_task.output).output + + @dsl.pipeline + def pythagorean(a: float = 1.2, b: float = 1.2) -> float: + sq_and_sum_task = square_and_sum(a=a, b=b) + make_float_task = convert_from_artifact( + dataset=sq_and_sum_task.output) + return square_root(x=make_float_task.output).output + + @dsl.pipeline + def pythagorean_then_add( + side: float, + addend: float = 42.24, + ) -> float: + t = pythagorean(a=side, b=1.2) + return add(x=t.output, y=addend).output + + task = pythagorean_then_add(side=2.2) + self.assertAlmostEqual(task.output, 44.745992817228334) + self.assert_output_dir_contents(1, 7) + + def test_parallel_for_not_supported(self): + local.init(local.SubprocessRunner()) + + @dsl.component + def pass_op(): + pass + + @dsl.pipeline + def my_pipeline(): + with dsl.ParallelFor([1, 2, 3]): + pass_op() + + with self.assertRaisesRegex( + NotImplementedError, + r"'dsl\.ParallelFor' is not supported by local pipeline execution\." + ): + my_pipeline() + + def test_condition_not_supported(self): + local.init(local.SubprocessRunner()) + + @dsl.component + def pass_op(): + pass + + @dsl.pipeline + def my_pipeline(x: str): + with dsl.Condition(x == 'foo'): + pass_op() + + with self.assertRaisesRegex( + NotImplementedError, + r"'dsl\.Condition' is not supported by local pipeline execution\." + ): + my_pipeline(x='bar') + + @mock.patch('sys.stdout', new_callable=stdlib_io.StringIO) + def test_fails_with_raise_on_error_true(self, mock_stdout): + local.init(local.SubprocessRunner(), raise_on_error=True) + + @dsl.component + def raise_component(): + raise Exception('Error from raise_component.') + + @dsl.pipeline + def my_pipeline(): + raise_component() + + with self.assertRaisesRegex( + RuntimeError, + r"Pipeline \x1b\[95m\'my-pipeline\'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\. Inner task failed: \x1b\[96m\'raise-component\'\x1b\[0m\.", + ): + my_pipeline() + + logged_output = mock_stdout.getvalue() + # Logs should: + # - log task failure trace + # - log pipeline failure + # - indicate which task the failure came from + self.assertRegex( + logged_output, + r"raise Exception\('Error from raise_component\.'\)", + ) + + @mock.patch('sys.stdout', new_callable=stdlib_io.StringIO) + def test_single_nested_fails_with_raise_on_error_true(self, mock_stdout): + local.init(local.SubprocessRunner(), raise_on_error=True) + + @dsl.component + def fail(): + raise Exception('Nested failure!') + + @dsl.pipeline + def inner_pipeline(): + fail() + + @dsl.pipeline + def my_pipeline(): + inner_pipeline() + + with self.assertRaisesRegex( + RuntimeError, + r"Pipeline \x1b\[95m\'my-pipeline\'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\. Inner task failed: \x1b\[96m\'fail\'\x1b\[0m inside \x1b\[96m\'inner-pipeline\'\x1b\[0m\.", + ): + my_pipeline() + + logged_output = mock_stdout.getvalue() + self.assertRegex( + logged_output, + r"raise Exception\('Nested failure!'\)", + ) + + @mock.patch('sys.stdout', new_callable=stdlib_io.StringIO) + def test_deeply_nested_fails_with_raise_on_error_true(self, mock_stdout): + local.init(local.SubprocessRunner(), raise_on_error=True) + + @dsl.component + def fail(): + raise Exception('Nested failure!') + + @dsl.pipeline + def deep_pipeline(): + fail() + + @dsl.pipeline + def mid_pipeline(): + deep_pipeline() + + @dsl.pipeline + def outer_pipeline(): + mid_pipeline() + + with self.assertRaisesRegex( + RuntimeError, + r"Pipeline \x1b\[95m\'outer-pipeline\'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\. Inner task failed: \x1b\[96m\'fail\'\x1b\[0m\ inside \x1b\[96m\'deep-pipeline\'\x1b\[0m inside \x1b\[96m\'mid-pipeline\'\x1b\[0m\.", + ): + outer_pipeline() + + logged_output = mock_stdout.getvalue() + self.assertRegex( + logged_output, + r"raise Exception\('Nested failure!'\)", + ) + + @mock.patch('sys.stdout', new_callable=stdlib_io.StringIO) + def test_fails_with_raise_on_error_false(self, mock_stdout): + local.init(local.SubprocessRunner(), raise_on_error=False) + + @dsl.component + def raise_component(): + raise Exception('Error from raise_component.') + + @dsl.pipeline + def my_pipeline(): + raise_component() + + task = my_pipeline() + logged_output = mock_stdout.getvalue() + # Logs should: + # - log task failure trace + # - log pipeline failure + # - indicate which task the failure came from + self.assertRegex( + logged_output, + r"raise Exception\('Error from raise_component\.'\)", + ) + self.assertRegex( + logged_output, + r"ERROR - Task \x1b\[96m'raise-component'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\n", + ) + self.assertRegex( + logged_output, + r"ERROR - Pipeline \x1b\[95m'my-pipeline'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\. Inner task failed: \x1b\[96m'raise-component'\x1b\[0m\.\n", + ) + self.assertEqual(task.outputs, {}) + + @mock.patch('sys.stdout', new_callable=stdlib_io.StringIO) + def test_single_nested_fails_with_raise_on_error_false(self, mock_stdout): + local.init(local.SubprocessRunner(), raise_on_error=False) + + @dsl.component + def fail(): + raise Exception('Nested failure!') + + @dsl.pipeline + def inner_pipeline(): + fail() + + @dsl.pipeline + def my_pipeline(): + inner_pipeline() + + task = my_pipeline() + logged_output = mock_stdout.getvalue() + # Logs should: + # - log task failure trace + # - log pipeline failure + # - indicate which task the failure came from + self.assertRegex( + logged_output, + r"raise Exception\('Nested failure!'\)", + ) + self.assertRegex( + logged_output, + r"ERROR - Task \x1b\[96m'fail'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\n", + ) + self.assertRegex( + logged_output, + r"ERROR - Pipeline \x1b\[95m\'my-pipeline\'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m. Inner task failed: \x1b\[96m\'fail\'\x1b\[0m inside \x1b\[96m\'inner-pipeline\'\x1b\[0m.\n" + ) + self.assertEqual(task.outputs, {}) + + @mock.patch('sys.stdout', new_callable=stdlib_io.StringIO) + def test_deeply_nested_fails_with_raise_on_error_false(self, mock_stdout): + local.init(local.SubprocessRunner(), raise_on_error=False) + + @dsl.component + def fail(): + raise Exception('Nested failure!') + + @dsl.pipeline + def deep_pipeline(): + fail() + + @dsl.pipeline + def mid_pipeline(): + deep_pipeline() + + @dsl.pipeline + def outer_pipeline(): + mid_pipeline() + + task = outer_pipeline() + logged_output = mock_stdout.getvalue() + # Logs should: + # - log task failure trace + # - log pipeline failure + # - indicate which task the failure came from + self.assertRegex( + logged_output, + r"raise Exception\('Nested failure!'\)", + ) + self.assertRegex( + logged_output, + r"ERROR - Task \x1b\[96m'fail'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\n", + ) + self.assertRegex( + logged_output, + r"ERROR - Pipeline \x1b\[95m'outer-pipeline'\x1b\[0m finished with status \x1b\[91mFAILURE\x1b\[0m\. Inner task failed: \x1b\[96m'fail'\x1b\[0m inside \x1b\[96m'deep-pipeline'\x1b\[0m inside \x1b\[96m'mid-pipeline'\x1b\[0m\.\n", + ) + self.assertEqual(task.outputs, {}) + + def test_fstring_python_component(self): + local.init(runner=local.SubprocessRunner()) + + @dsl.component + def identity(string: str) -> str: + return string + + @dsl.pipeline + def my_pipeline(string: str = 'baz') -> str: + op1 = identity(string=f'bar-{string}') + op2 = identity(string=f'foo-{op1.output}') + return op2.output + + task = my_pipeline() + self.assertEqual(task.output, 'foo-bar-baz') + + +class TestFstringContainerComponent( + testing_utilities.LocalRunnerEnvironmentTestCase): + + @classmethod + def setUpClass(cls): + from kfp.local import subprocess_task_handler + + # Temporarily removing these these validation calls is useful hack to + # test a ContainerComponent outside of a container. + # We do this here because we only want to test the very specific + # f-string logic in container components without the presence of + # Docker in the test environment. + cls.original_validate_image = subprocess_task_handler.SubprocessTaskHandler.validate_image + subprocess_task_handler.SubprocessTaskHandler.validate_image = lambda slf, image: None + + cls.original_validate_not_container_component = subprocess_task_handler.SubprocessTaskHandler.validate_not_container_component + subprocess_task_handler.SubprocessTaskHandler.validate_not_container_component = lambda slf, full_command: None + + cls.original_validate_not_containerized_python_component = subprocess_task_handler.SubprocessTaskHandler.validate_not_containerized_python_component + subprocess_task_handler.SubprocessTaskHandler.validate_not_containerized_python_component = lambda slf, full_command: None + + @classmethod + def tearDownClass(cls): + from kfp.local import subprocess_task_handler + + subprocess_task_handler.SubprocessTaskHandler.validate_image = cls.original_validate_image + subprocess_task_handler.SubprocessTaskHandler.validate_not_container_component = cls.original_validate_not_container_component + subprocess_task_handler.SubprocessTaskHandler.validate_not_containerized_python_component = cls.original_validate_not_containerized_python_component + + def test_fstring_container_component(self): + local.init(runner=local.SubprocessRunner()) + + @dsl.container_component + def identity_container(string: str, outpath: dsl.OutputPath(str)): + return dsl.ContainerSpec( + image='alpine', + command=[ + 'sh', + '-c', + f"""mkdir -p $(dirname {outpath}) && printf '%s' {string} > {outpath}""", + ]) + + @dsl.pipeline + def my_pipeline(string: str = 'baz') -> str: + op1 = identity_container(string=f'bar-{string}') + op2 = identity_container(string=f'foo-{op1.output}') + return op2.output + + task = my_pipeline() + self.assertEqual(task.output, 'foo-bar-baz') + + +if __name__ == '__main__': + unittest.main() diff --git a/sdk/python/kfp/local/placeholder_utils.py b/sdk/python/kfp/local/placeholder_utils.py index 3333fff6e5..405b48e851 100644 --- a/sdk/python/kfp/local/placeholder_utils.py +++ b/sdk/python/kfp/local/placeholder_utils.py @@ -12,6 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. """Utilities for working with placeholders.""" +import functools import json import random import re @@ -20,8 +21,8 @@ from kfp import dsl -def make_random_id(): - """Makes a random 8 digit integer.""" +def make_random_id() -> str: + """Makes a random 8 digit integer as a string.""" return str(random.randint(0, 99999999)) @@ -31,10 +32,24 @@ def replace_placeholders( pipeline_resource_name: str, task_resource_name: str, pipeline_root: str, + unique_pipeline_id: str, ) -> List[str]: - """Iterates over each element in the command and replaces placeholders.""" - unique_pipeline_id = make_random_id() + """Iterates over each element in the command and replaces placeholders. + + This should only be called once per each task, since the task's + random ID is created within the scope of the function. Multiple + calls on the same task will result in multiple random IDs per single + task. + """ unique_task_id = make_random_id() + executor_input_dict = resolve_self_references_in_executor_input( + executor_input_dict=executor_input_dict, + pipeline_resource_name=pipeline_resource_name, + task_resource_name=task_resource_name, + pipeline_root=pipeline_root, + pipeline_job_id=unique_pipeline_id, + pipeline_task_id=unique_task_id, + ) provided_inputs = get_provided_inputs(executor_input_dict) full_command = [ resolve_struct_placeholders( @@ -67,6 +82,88 @@ def replace_placeholders( return resolved_command +def resolve_self_references_in_executor_input( + executor_input_dict: Dict[str, Any], + pipeline_resource_name: str, + task_resource_name: str, + pipeline_root: str, + pipeline_job_id: str, + pipeline_task_id: str, +) -> Dict[str, Any]: + """Resolve parameter placeholders that point to other parameter + placeholders in the same ExecutorInput message. + + This occurs when passing f-strings to a component. For example: + + my_comp(foo=f'bar-{upstream.output}') + + May result in the ExecutorInput message: + + {'inputs': {'parameterValues': {'pipelinechannel--identity-Output': 'foo', + 'string': "{{$.inputs.parameters['pipelinechannel--identity-Output']}}-bar"}}, + 'outputs': ...} + + The placeholder "{{$.inputs.parameters['pipelinechannel--identity-Output']}}-bar" points to parameter 'pipelinechannel--identity-Output' with the value 'foo'. This function replaces "{{$.inputs.parameters['pipelinechannel--identity-Output']}}-bar" with 'foo'. + """ + for k, v in executor_input_dict.get('inputs', + {}).get('parameterValues', {}).items(): + if isinstance(v, str): + executor_input_dict['inputs']['parameterValues'][ + k] = resolve_individual_placeholder( + v, + executor_input_dict=executor_input_dict, + pipeline_resource_name=pipeline_resource_name, + task_resource_name=task_resource_name, + pipeline_root=pipeline_root, + pipeline_job_id=pipeline_job_id, + pipeline_task_id=pipeline_task_id, + ) + return executor_input_dict + + +def recursively_resolve_json_dict_placeholders( + obj: Any, + executor_input_dict: Dict[str, Any], + pipeline_resource_name: str, + task_resource_name: str, + pipeline_root: str, + pipeline_job_id: str, + pipeline_task_id: str, +) -> Any: + """Recursively resolves any placeholders in a dictionary representation of + a JSON object. + + These objects are very unlikely to be sufficiently large to exceed + max recursion depth of 1000 and an iterative implementation is much + less readable, so preferring recursive implementation. + """ + inner_fn = functools.partial( + recursively_resolve_json_dict_placeholders, + executor_input_dict=executor_input_dict, + pipeline_resource_name=pipeline_resource_name, + task_resource_name=task_resource_name, + pipeline_root=pipeline_root, + pipeline_job_id=pipeline_job_id, + pipeline_task_id=pipeline_task_id, + ) + if isinstance(obj, list): + return [inner_fn(item) for item in obj] + elif isinstance(obj, dict): + return {inner_fn(key): inner_fn(value) for key, value in obj.items()} + elif isinstance(obj, str): + return resolve_individual_placeholder( + element=obj, + executor_input_dict=executor_input_dict, + pipeline_resource_name=pipeline_resource_name, + task_resource_name=task_resource_name, + pipeline_root=pipeline_root, + pipeline_job_id=pipeline_job_id, + pipeline_task_id=pipeline_task_id, + ) + else: + return obj + + def flatten_list(l: List[Union[str, list, None]]) -> List[str]: """Iteratively flattens arbitrarily deeply nested lists, filtering out elements that are None.""" @@ -133,6 +230,10 @@ def resolve_io_placeholders( executor_input: Dict[str, Any], command: str, ) -> str: + """Resolves placeholders in command using executor_input. + + executor_input should not contain any unresolved placeholders. + """ placeholders = re.findall(r'\{\{\$\.(.*?)\}\}', command) # e.g., placeholder = "inputs.parameters[''text'']" diff --git a/sdk/python/kfp/local/placeholder_utils_test.py b/sdk/python/kfp/local/placeholder_utils_test.py index 7ecd71dfa0..05f83c2d27 100644 --- a/sdk/python/kfp/local/placeholder_utils_test.py +++ b/sdk/python/kfp/local/placeholder_utils_test.py @@ -31,7 +31,26 @@ 'dictionary': { 'foo': 'bar' }, - } + }, + 'artifacts': { + 'in_a': { + 'artifacts': [{ + 'name': + 'in_a', + 'type': { + 'schemaTitle': 'system.Dataset', + 'schemaVersion': '0.0.1' + }, + 'uri': + '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/upstream-comp/in_a', + 'metadata': { + 'foo': { + 'bar': 'baz' + } + } + }] + } + }, }, 'outputs': { 'parameters': { @@ -51,6 +70,12 @@ }, 'uri': '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/comp/out_a', + # include metadata on outputs since it allows us to + # test the placeholder + # "{{$.outputs.artifacts[''out_a''].metadata[''foo'']}}" + # for comprehensive testing, but in practice metadata + # will never be set on output artifacts since they + # haven't been created yet 'metadata': { 'foo': { 'bar': 'baz' @@ -62,7 +87,8 @@ 'outputFile': '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/comp/executor_output.json' } - }, executor_input) + }, + executor_input) EXECUTOR_INPUT_DICT = json_format.MessageToDict(executor_input) @@ -83,6 +109,7 @@ def test(self): pipeline_resource_name='my-pipeline-2023-10-10-13-32-59-420710', task_resource_name='comp', pipeline_root='/foo/bar/my-pipeline-2023-10-10-13-32-59-420710', + unique_pipeline_id=placeholder_utils.make_random_id(), ) expected = [ 'echo', @@ -96,7 +123,6 @@ def test(self): class TestResolveIndividualPlaceholder(parameterized.TestCase): # TODO: consider supporting JSON escape - # TODO: update when input artifact constants supported # TODO: update when output lists of artifacts are supported @parameterized.parameters([ ( @@ -202,6 +228,24 @@ def test_concatenated_placeholders_resolve(self, element: str, "{{$.outputs.artifacts[''out_a''].metadata[''foo'']}}", json.dumps({'bar': 'baz'}), ), + ( + "{{$.inputs.artifacts[''in_a''].metadata}}", + json.dumps({'foo': { + 'bar': 'baz' + }}), + ), + ( + "{{$.inputs.artifacts[''in_a''].uri}}", + '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/upstream-comp/in_a', + ), + ( + "{{$.inputs.artifacts[''in_a''].path}}", + '/foo/bar/my-pipeline-2023-10-10-13-32-59-420710/upstream-comp/in_a', + ), + ( + "{{$.inputs.artifacts[''in_a''].metadata[''foo'']}}", + json.dumps({'bar': 'baz'}), + ), ]) def test_io_placeholders(self, element: str, expected: str): actual = placeholder_utils.resolve_individual_placeholder( @@ -374,5 +418,45 @@ def test( self.assertEqual(actual, expected) +class TestResolveSelfReferencesInExecutorInput(unittest.TestCase): + + def test_simple(self): + executor_input_dict = { + 'inputs': { + 'parameterValues': { + 'pipelinechannel--identity-Output': + 'foo', + 'string': + "{{$.inputs.parameters['pipelinechannel--identity-Output']}}-bar" + } + }, + 'outputs': { + 'outputFile': + '/foo/bar/my-pipeline-2024-01-26-12-26-24-162075/echo/executor_output.json' + } + } + expected = { + 'inputs': { + 'parameterValues': { + 'pipelinechannel--identity-Output': 'foo', + 'string': 'foo-bar' + } + }, + 'outputs': { + 'outputFile': + '/foo/bar/my-pipeline-2024-01-26-12-26-24-162075/echo/executor_output.json' + } + } + actual = placeholder_utils.resolve_self_references_in_executor_input( + executor_input_dict, + pipeline_resource_name='my-pipeline-2024-01-26-12-26-24-162075', + task_resource_name='echo', + pipeline_root='/foo/bar/my-pipeline-2024-01-26-12-26-24-162075', + pipeline_job_id='123456789', + pipeline_task_id='987654321', + ) + self.assertEqual(actual, expected) + + if __name__ == '__main__': unittest.main() diff --git a/sdk/python/kfp/local/task_dispatcher.py b/sdk/python/kfp/local/task_dispatcher.py index f051f64b9f..047bcd92b8 100755 --- a/sdk/python/kfp/local/task_dispatcher.py +++ b/sdk/python/kfp/local/task_dispatcher.py @@ -13,7 +13,7 @@ # limitations under the License. """Code for dispatching a local task execution.""" import logging -from typing import Any, Dict +from typing import Any, Dict, Tuple from kfp import local from kfp.local import config @@ -25,10 +25,11 @@ from kfp.local import status from kfp.local import subprocess_task_handler from kfp.local import task_handler_interface +from kfp.local import utils from kfp.pipeline_spec import pipeline_spec_pb2 -def run_single_component( +def run_single_task( pipeline_spec: pipeline_spec_pb2.PipelineSpec, arguments: Dict[str, Any], ) -> Dict[str, Any]: @@ -41,34 +42,60 @@ def run_single_component( Returns: A LocalTask instance. """ - if config.LocalExecutionConfig.instance is None: - raise RuntimeError( - f"Local environment not initialized. Please run '{local.__name__}.{local.init.__name__}()' before executing tasks locally." - ) + config.LocalExecutionConfig.validate() + component_name, component_spec = list(pipeline_spec.components.items())[0] + executor_spec = get_executor_spec( + pipeline_spec, + component_spec.executor_label, + ) + executor_spec = utils.struct_to_executor_spec(executor_spec) + pipeline_resource_name = executor_input_utils.get_local_pipeline_resource_name( + pipeline_spec.pipeline_info.name) + # all global state should be accessed here # do not access local config state downstream - return _run_single_component_implementation( - pipeline_spec=pipeline_spec, + outputs, _ = run_single_task_implementation( + pipeline_resource_name=pipeline_resource_name, + component_name=component_name, + component_spec=component_spec, + executor_spec=executor_spec, arguments=arguments, pipeline_root=config.LocalExecutionConfig.instance.pipeline_root, runner=config.LocalExecutionConfig.instance.runner, raise_on_error=config.LocalExecutionConfig.instance.raise_on_error, - ) + block_input_artifact=True, + unique_pipeline_id=placeholder_utils.make_random_id()) + return outputs -def _run_single_component_implementation( +def get_executor_spec( pipeline_spec: pipeline_spec_pb2.PipelineSpec, + executor_label: str, +) -> pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec: + return pipeline_spec.deployment_spec['executors'][executor_label] + + +Outputs = Dict[str, Any] + + +def run_single_task_implementation( + pipeline_resource_name: str, + component_name: str, + component_spec: pipeline_spec_pb2.ComponentSpec, + executor_spec: pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec, arguments: Dict[str, Any], pipeline_root: str, runner: config.LocalRunnerType, raise_on_error: bool, -) -> Dict[str, Any]: - """The implementation of a single component runner.""" + block_input_artifact: bool, + unique_pipeline_id: str, +) -> Tuple[Outputs, status.Status]: + """The implementation of a single component runner. - component_name, component_spec = list(pipeline_spec.components.items())[0] + Returns a tuple of (outputs, status). If status is FAILURE, outputs + is an empty dictionary. + """ - pipeline_resource_name = executor_input_utils.get_local_pipeline_resource_name( - pipeline_spec.pipeline_info.name) task_resource_name = executor_input_utils.get_local_task_resource_name( component_name) task_root = executor_input_utils.construct_local_task_root( @@ -80,17 +107,12 @@ def _run_single_component_implementation( component_spec=component_spec, arguments=arguments, task_root=task_root, + block_input_artifact=block_input_artifact, ) - executor_spec = pipeline_spec.deployment_spec['executors'][ - component_spec.executor_label] - - container = executor_spec['container'] - image = container['image'] - - command = list(container['command']) if 'command' in container else [] - args = list(container['args']) if 'args' in container else [] - full_command = command + args + container = executor_spec.container + image = container.image + full_command = list(container.command) + list(container.args) executor_input_dict = executor_input_utils.executor_input_to_dict( executor_input=executor_input, @@ -102,6 +124,7 @@ def _run_single_component_implementation( pipeline_resource_name=pipeline_resource_name, task_resource_name=task_resource_name, pipeline_root=pipeline_root, + unique_pipeline_id=unique_pipeline_id, ) runner_type = type(runner) @@ -115,10 +138,7 @@ def _run_single_component_implementation( TaskHandler = task_handler_map[runner_type] with logging_utils.local_logger_context(): - task_name_for_logs = logging_utils.color_text( - f'{task_resource_name!r}', - logging_utils.Color.CYAN, - ) + task_name_for_logs = logging_utils.format_task_name(task_resource_name) logging.info(f'Executing task {task_name_for_logs}') task_handler = TaskHandler( @@ -137,7 +157,7 @@ def _run_single_component_implementation( if task_status == status.Status.SUCCESS: logging.info( - f'Task {task_name_for_logs} finished with status {logging_utils.color_text(task_status.value, logging_utils.Color.GREEN)}' + f'Task {task_name_for_logs} finished with status {logging_utils.format_status(task_status)}' ) outputs = executor_output_utils.get_outputs_for_task( @@ -148,14 +168,13 @@ def _run_single_component_implementation( output_string = [ f'Task {task_name_for_logs} outputs:', *logging_utils.make_log_lines_for_outputs(outputs), - '\n', ] logging.info('\n'.join(output_string)) else: logging.info(f'Task {task_name_for_logs} has no outputs') elif task_status == status.Status.FAILURE: - msg = f'Task {task_name_for_logs} finished with status {logging_utils.color_text(task_status.value, logging_utils.Color.RED)}' + msg = f'Task {task_name_for_logs} finished with status {logging_utils.format_status(task_status)}' if raise_on_error: raise RuntimeError(msg) else: @@ -165,5 +184,6 @@ def _run_single_component_implementation( else: # for developers; user should never hit this raise ValueError(f'Got unknown status: {task_status}') + logging_utils.print_horizontal_line() - return outputs + return outputs, task_status diff --git a/sdk/python/kfp/local/task_dispatcher_test.py b/sdk/python/kfp/local/task_dispatcher_test.py index bd086e25ad..d8163aab89 100755 --- a/sdk/python/kfp/local/task_dispatcher_test.py +++ b/sdk/python/kfp/local/task_dispatcher_test.py @@ -114,63 +114,6 @@ def artifact_identity(a: Artifact) -> Artifact: class TestSupportOfComponentTypes( testing_utilities.LocalRunnerEnvironmentTestCase): - def test_local_pipeline_unsupported_two_tasks(self): - local.init(runner=local.SubprocessRunner(use_venv=True)) - - @dsl.component - def identity(x: str) -> str: - return x - - @dsl.pipeline - def my_pipeline(): - identity(x='foo') - identity(x='bar') - - # compile and load into a YamlComponent to ensure the NotImplementedError isn't simply being thrown because this is a GraphComponent - my_pipeline = testing_utilities.compile_and_load_component(my_pipeline) - with self.assertRaisesRegex( - NotImplementedError, - r'Local pipeline execution is not currently supported\.', - ): - my_pipeline() - - def test_local_pipeline_unsupported_one_task_different_interface(self): - local.init(runner=local.SubprocessRunner(use_venv=True)) - - @dsl.component - def identity(x: str) -> str: - return x - - @dsl.pipeline - def my_pipeline(): - identity(x='foo') - - # compile and load into a YamlComponent to ensure the NotImplementedError isn't simply being thrown because this is a GraphComponent - my_pipeline = testing_utilities.compile_and_load_component(my_pipeline) - with self.assertRaisesRegex( - NotImplementedError, - r'Local pipeline execution is not currently supported\.', - ): - my_pipeline() - - def test_local_pipeline_unsupported_if_is_graph_component(self): - local.init(runner=local.SubprocessRunner(use_venv=True)) - - @dsl.component - def identity(x: str) -> str: - return x - - # even if there is one task with the same interface as the pipeline, the code should catch that the pipeline is a GraphComponent and throw the NotImplementedError - @dsl.pipeline - def my_pipeline(string: str) -> str: - return identity(x=string).output - - with self.assertRaisesRegex( - NotImplementedError, - r'Local pipeline execution is not currently supported\.', - ): - my_pipeline(string='foo') - def test_can_run_loaded_component(self): # use venv to avoid installing non-local KFP into test process local.init(runner=local.SubprocessRunner(use_venv=True)) @@ -271,7 +214,7 @@ def many_type_component( r'Wrote executor output file to', r'.*', r"\d+:\d+:\d+\.\d+ - INFO - Task \x1b\[96m'many-type-component'\x1b\[0m finished with status \x1b\[92mSUCCESS\x1b\[0m\n", - r"\d+:\d+:\d+\.\d+ - INFO - Task \x1b\[96m'many-type-component'\x1b\[0m outputs:\n Output: 'hellohello'\n model: Model\( name='model',\n uri='[a-zA-Z0-9/_\.-]+/local_outputs/many-type-component-\d+-\d+-\d+-\d+-\d+-\d+-\d+/many-type-component/model',\n metadata={'foo': 'bar'} \)\n\n", + r"\d+:\d+:\d+\.\d+ - INFO - Task \x1b\[96m'many-type-component'\x1b\[0m outputs:\n Output: 'hellohello'\n model: Model\( name='model',\n uri='[a-zA-Z0-9/_\.-]+/local_outputs/many-type-component-\d+-\d+-\d+-\d+-\d+-\d+-\d+/many-type-component/model',\n metadata={'foo': 'bar'} \)\n", ] self.assertRegex( diff --git a/sdk/python/kfp/local/utils.py b/sdk/python/kfp/local/utils.py new file mode 100644 index 0000000000..7be5a2a970 --- /dev/null +++ b/sdk/python/kfp/local/utils.py @@ -0,0 +1,26 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Assorted utilities.""" + +from google.protobuf import json_format +from google.protobuf import struct_pb2 +from kfp.pipeline_spec import pipeline_spec_pb2 + + +def struct_to_executor_spec( + struct: struct_pb2.Struct, +) -> pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec: + executor_spec = pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec() + json_format.ParseDict(json_format.MessageToDict(struct), executor_spec) + return executor_spec diff --git a/sdk/python/kfp/local/utils_test.py b/sdk/python/kfp/local/utils_test.py new file mode 100644 index 0000000000..d7b1698792 --- /dev/null +++ b/sdk/python/kfp/local/utils_test.py @@ -0,0 +1,45 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +"""Test for utils.py.""" +import unittest + +from google.protobuf import json_format +from google.protobuf import struct_pb2 +from kfp.local import utils +from kfp.pipeline_spec import pipeline_spec_pb2 + + +class TestDictToExecutorSpec(unittest.TestCase): + + def test_simple(self): + input_struct = struct_pb2.Struct() + input_dict = { + 'container': { + 'image': 'alpine', + 'command': ['echo'], + 'args': ['foo'], + } + } + json_format.ParseDict(input_dict, input_struct) + expected = pipeline_spec_pb2.PipelineDeploymentConfig.ExecutorSpec( + container=pipeline_spec_pb2.PipelineDeploymentConfig + .PipelineContainerSpec( + image='alpine', command=['echo'], args=['foo'])) + + actual = utils.struct_to_executor_spec(input_struct) + self.assertEqual(actual, expected) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdk/python/test_data/pipelines/pipeline_with_parallelfor_list_artifacts.py b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_list_artifacts.py new file mode 100644 index 0000000000..64ab75d31d --- /dev/null +++ b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_list_artifacts.py @@ -0,0 +1,77 @@ +# Copyright 2024 The Kubeflow Authors +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from typing import List + +from kfp import compiler +from kfp import dsl +from kfp.dsl import Artifact +from kfp.dsl import Dataset + + +@dsl.component +def print_artifact_name(artifact: Artifact) -> str: + print(artifact.name) + return artifact.name + + +@dsl.component +def make_dataset(data: str) -> Dataset: + dataset = Dataset(uri=dsl.get_uri(), metadata={'length': len(data)}) + with open(dataset.path, 'w') as f: + f.write(data) + return dataset + + +@dsl.pipeline +def make_datasets( + texts: List[str] = ['Hello', ',', ' ', 'world!']) -> List[Dataset]: + with dsl.ParallelFor(texts) as text: + t1 = make_dataset(data=text) + + return dsl.Collected(t1.output) + + +@dsl.component +def make_artifact(data: str) -> Artifact: + artifact = Artifact(uri=dsl.get_uri(), metadata={'length': len(data)}) + with open(artifact.path, 'w') as f: + f.write(data) + return artifact + + +@dsl.pipeline +def make_artifacts( + texts: List[str] = ['Hello', ',', ' ', 'world!']) -> List[Artifact]: + with dsl.ParallelFor(texts) as text: + t1 = make_artifact(data=text) + + return dsl.Collected(t1.output) + + +@dsl.pipeline(name='pipeline-parallelfor-artifacts') +def my_pipeline(): + make_artifacts_task = make_artifacts() + with dsl.ParallelFor(items=make_artifacts_task.output) as item: + print_artifact_name(artifact=item) + + make_datasets_task = make_datasets() + with dsl.ParallelFor(items=make_datasets_task.output) as item: + print_artifact_name(artifact=item) + + +if __name__ == '__main__': + compiler.Compiler().compile( + pipeline_func=my_pipeline, + package_path=__file__.replace('.py', '.yaml')) diff --git a/sdk/python/test_data/pipelines/pipeline_with_parallelfor_list_artifacts.yaml b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_list_artifacts.yaml new file mode 100644 index 0000000000..93a4efd716 --- /dev/null +++ b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_list_artifacts.yaml @@ -0,0 +1,420 @@ +# PIPELINE DEFINITION +# Name: pipeline-parallelfor-artifacts +components: + comp-for-loop-1: + dag: + tasks: + print-artifact-name: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-artifact-name + inputs: + artifacts: + artifact: + componentInputArtifact: pipelinechannel--make-artifacts-Output-loop-item + taskInfo: + name: print-artifact-name + inputDefinitions: + artifacts: + pipelinechannel--make-artifacts-Output: + artifactType: + schemaTitle: system.Artifact + schemaVersion: 0.0.1 + isArtifactList: true + pipelinechannel--make-artifacts-Output-loop-item: + artifactType: + schemaTitle: system.Artifact + schemaVersion: 0.0.1 + comp-for-loop-1-2: + dag: + outputs: + artifacts: + pipelinechannel--make-artifact-Output: + artifactSelectors: + - outputArtifactKey: Output + producerSubtask: make-artifact + tasks: + make-artifact: + cachingOptions: + enableCache: true + componentRef: + name: comp-make-artifact + inputs: + parameters: + data: + componentInputParameter: pipelinechannel--texts-loop-item + taskInfo: + name: make-artifact + inputDefinitions: + parameters: + pipelinechannel--texts: + parameterType: LIST + pipelinechannel--texts-loop-item: + parameterType: STRING + outputDefinitions: + artifacts: + pipelinechannel--make-artifact-Output: + artifactType: + schemaTitle: system.Artifact + schemaVersion: 0.0.1 + isArtifactList: true + comp-for-loop-1-3: + dag: + outputs: + artifacts: + pipelinechannel--make-dataset-Output: + artifactSelectors: + - outputArtifactKey: Output + producerSubtask: make-dataset + tasks: + make-dataset: + cachingOptions: + enableCache: true + componentRef: + name: comp-make-dataset + inputs: + parameters: + data: + componentInputParameter: pipelinechannel--texts-loop-item + taskInfo: + name: make-dataset + inputDefinitions: + parameters: + pipelinechannel--texts: + parameterType: LIST + pipelinechannel--texts-loop-item: + parameterType: STRING + outputDefinitions: + artifacts: + pipelinechannel--make-dataset-Output: + artifactType: + schemaTitle: system.Dataset + schemaVersion: 0.0.1 + isArtifactList: true + comp-for-loop-2: + dag: + tasks: + print-artifact-name-2: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-artifact-name-2 + inputs: + artifacts: + artifact: + componentInputArtifact: pipelinechannel--make-datasets-Output-loop-item + taskInfo: + name: print-artifact-name-2 + inputDefinitions: + artifacts: + pipelinechannel--make-datasets-Output: + artifactType: + schemaTitle: system.Dataset + schemaVersion: 0.0.1 + isArtifactList: true + pipelinechannel--make-datasets-Output-loop-item: + artifactType: + schemaTitle: system.Dataset + schemaVersion: 0.0.1 + comp-make-artifact: + executorLabel: exec-make-artifact + inputDefinitions: + parameters: + data: + parameterType: STRING + outputDefinitions: + artifacts: + Output: + artifactType: + schemaTitle: system.Artifact + schemaVersion: 0.0.1 + comp-make-artifacts: + dag: + outputs: + artifacts: + Output: + artifactSelectors: + - outputArtifactKey: pipelinechannel--make-artifact-Output + producerSubtask: for-loop-1 + tasks: + for-loop-1: + componentRef: + name: comp-for-loop-1-2 + inputs: + parameters: + pipelinechannel--texts: + componentInputParameter: texts + parameterIterator: + itemInput: pipelinechannel--texts-loop-item + items: + inputParameter: pipelinechannel--texts + taskInfo: + name: for-loop-1 + inputDefinitions: + parameters: + texts: + defaultValue: + - Hello + - ',' + - ' ' + - world! + isOptional: true + parameterType: LIST + outputDefinitions: + artifacts: + Output: + artifactType: + schemaTitle: system.Artifact + schemaVersion: 0.0.1 + isArtifactList: true + comp-make-dataset: + executorLabel: exec-make-dataset + inputDefinitions: + parameters: + data: + parameterType: STRING + outputDefinitions: + artifacts: + Output: + artifactType: + schemaTitle: system.Dataset + schemaVersion: 0.0.1 + comp-make-datasets: + dag: + outputs: + artifacts: + Output: + artifactSelectors: + - outputArtifactKey: pipelinechannel--make-dataset-Output + producerSubtask: for-loop-1 + tasks: + for-loop-1: + componentRef: + name: comp-for-loop-1-3 + inputs: + parameters: + pipelinechannel--texts: + componentInputParameter: texts + parameterIterator: + itemInput: pipelinechannel--texts-loop-item + items: + inputParameter: pipelinechannel--texts + taskInfo: + name: for-loop-1 + inputDefinitions: + parameters: + texts: + defaultValue: + - Hello + - ',' + - ' ' + - world! + isOptional: true + parameterType: LIST + outputDefinitions: + artifacts: + Output: + artifactType: + schemaTitle: system.Dataset + schemaVersion: 0.0.1 + isArtifactList: true + comp-print-artifact-name: + executorLabel: exec-print-artifact-name + inputDefinitions: + artifacts: + artifact: + artifactType: + schemaTitle: system.Artifact + schemaVersion: 0.0.1 + outputDefinitions: + parameters: + Output: + parameterType: STRING + comp-print-artifact-name-2: + executorLabel: exec-print-artifact-name-2 + inputDefinitions: + artifacts: + artifact: + artifactType: + schemaTitle: system.Artifact + schemaVersion: 0.0.1 + outputDefinitions: + parameters: + Output: + parameterType: STRING +deploymentSpec: + executors: + exec-make-artifact: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - make_artifact + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef make_artifact(data: str) -> Artifact:\n artifact = Artifact(uri=dsl.get_uri(),\ + \ metadata={'length': len(data)})\n with open(artifact.path, 'w') as\ + \ f:\n f.write(data)\n return artifact\n\n" + image: python:3.7 + exec-make-dataset: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - make_dataset + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef make_dataset(data: str) -> Dataset:\n dataset = Dataset(uri=dsl.get_uri(),\ + \ metadata={'length': len(data)})\n with open(dataset.path, 'w') as f:\n\ + \ f.write(data)\n return dataset\n\n" + image: python:3.7 + exec-print-artifact-name: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_artifact_name + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_artifact_name(artifact: Artifact) -> str:\n print(artifact.name)\n\ + \ return artifact.name\n\n" + image: python:3.7 + exec-print-artifact-name-2: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_artifact_name + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_artifact_name(artifact: Artifact) -> str:\n print(artifact.name)\n\ + \ return artifact.name\n\n" + image: python:3.7 +pipelineInfo: + name: pipeline-parallelfor-artifacts +root: + dag: + tasks: + for-loop-1: + artifactIterator: + itemInput: pipelinechannel--make-artifacts-Output-loop-item + items: + inputArtifact: pipelinechannel--make-artifacts-Output + componentRef: + name: comp-for-loop-1 + dependentTasks: + - make-artifacts + inputs: + artifacts: + pipelinechannel--make-artifacts-Output: + taskOutputArtifact: + outputArtifactKey: Output + producerTask: make-artifacts + taskInfo: + name: for-loop-1 + for-loop-2: + artifactIterator: + itemInput: pipelinechannel--make-datasets-Output-loop-item + items: + inputArtifact: pipelinechannel--make-datasets-Output + componentRef: + name: comp-for-loop-2 + dependentTasks: + - make-datasets + inputs: + artifacts: + pipelinechannel--make-datasets-Output: + taskOutputArtifact: + outputArtifactKey: Output + producerTask: make-datasets + taskInfo: + name: for-loop-2 + make-artifacts: + cachingOptions: + enableCache: true + componentRef: + name: comp-make-artifacts + taskInfo: + name: make-artifacts + make-datasets: + cachingOptions: + enableCache: true + componentRef: + name: comp-make-datasets + taskInfo: + name: make-datasets +schemaVersion: 2.1.0 +sdkVersion: kfp-2.6.0 diff --git a/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.py b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.py index b8e52eb79e..f477767dd6 100644 --- a/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.py +++ b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.py @@ -12,9 +12,12 @@ # See the License for the specific language governing permissions and # limitations under the License. -from typing import List +import os +import tempfile +from typing import Dict, List from kfp import compiler +from kfp import components from kfp import dsl from kfp.dsl import component @@ -24,6 +27,58 @@ def print_text(msg: str): print(msg) +@component +def print_int(x: int): + print(x) + + +@component +def list_dict_maker_0() -> List[Dict[str, int]]: + """Enforces strict type checking - returns a list of dictionaries + where keys are strings and values are integers. For testing type + handling during compilation.""" + return [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}] + + +@component +def list_dict_maker_1() -> List[Dict]: + """Utilizes generic dictionary typing (no enforcement of specific key or + value types). + + Tests flexibility in type handling. + """ + return [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}] + + +@component +def list_dict_maker_2() -> List[dict]: + """Returns a list of dictionaries without type enforcement. + + Tests flexibility in type handling. + """ + return [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}] + + +@component +def list_dict_maker_3() -> List: + """Returns a basic list (no typing or structure guarantees). + + Tests the limits of compiler type handling. + """ + return [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}] + + +with tempfile.TemporaryDirectory() as tmpdir: + pipeline_package_path = os.path.join(tmpdir, 'upstream_component.yaml') + compiler.Compiler().compile( + pipeline_func=list_dict_maker_1, + package_path=pipeline_package_path, + ) + + loaded_dict_maker = components.load_component_from_file( + pipeline_package_path) + + @dsl.pipeline(name='pipeline-with-loops') def my_pipeline(loop_parameter: List[str]): @@ -52,6 +107,33 @@ def my_pipeline(loop_parameter: List[str]): print_text(msg=nested_item.A_a) print_text(msg=nested_item.B_b) + # Loop argument that is a static dictionary known at compile time. + dict_loop_argument = [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}] + with dsl.ParallelFor(items=dict_loop_argument, parallelism=1) as item: + print_int(x=item.a) + + # Loop argument that coming from the upstream component. + t_0 = list_dict_maker_0() + with dsl.ParallelFor(items=t_0.output) as item: + print_int(x=item.a) + + t_1 = list_dict_maker_1() + with dsl.ParallelFor(items=t_1.output) as item: + print_int(x=item.a) + + t_2 = list_dict_maker_2() + with dsl.ParallelFor(items=t_2.output) as item: + print_int(x=item.a) + + t_3 = list_dict_maker_3() + with dsl.ParallelFor(items=t_3.output) as item: + print_int(x=item.a) + + # Loop argument that coming from the upstream component compiled file. + t_4 = loaded_dict_maker() + with dsl.ParallelFor(items=t_4.output) as item: + print_int(x=item.a) + if __name__ == '__main__': compiler.Compiler().compile( diff --git a/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.yaml b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.yaml index eaac51c057..2d716ac4c7 100644 --- a/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.yaml +++ b/sdk/python/test_data/pipelines/pipeline_with_parallelfor_parallelism.yaml @@ -36,6 +36,90 @@ components: parameterType: LIST pipelinechannel--loop_parameter-loop-item: parameterType: STRING + comp-for-loop-10: + dag: + tasks: + print-int-3: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-int-3 + inputs: + parameters: + x: + componentInputParameter: pipelinechannel--list-dict-maker-1-Output-loop-item + parameterExpressionSelector: parseJson(string_value)["a"] + taskInfo: + name: print-int-3 + inputDefinitions: + parameters: + pipelinechannel--list-dict-maker-1-Output: + parameterType: LIST + pipelinechannel--list-dict-maker-1-Output-loop-item: + parameterType: STRUCT + comp-for-loop-11: + dag: + tasks: + print-int-4: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-int-4 + inputs: + parameters: + x: + componentInputParameter: pipelinechannel--list-dict-maker-2-Output-loop-item + parameterExpressionSelector: parseJson(string_value)["a"] + taskInfo: + name: print-int-4 + inputDefinitions: + parameters: + pipelinechannel--list-dict-maker-2-Output: + parameterType: LIST + pipelinechannel--list-dict-maker-2-Output-loop-item: + parameterType: STRUCT + comp-for-loop-12: + dag: + tasks: + print-int-5: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-int-5 + inputs: + parameters: + x: + componentInputParameter: pipelinechannel--list-dict-maker-3-Output-loop-item + parameterExpressionSelector: parseJson(string_value)["a"] + taskInfo: + name: print-int-5 + inputDefinitions: + parameters: + pipelinechannel--list-dict-maker-3-Output: + parameterType: LIST + pipelinechannel--list-dict-maker-3-Output-loop-item: + parameterType: STRING + comp-for-loop-13: + dag: + tasks: + print-int-6: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-int-6 + inputs: + parameters: + x: + componentInputParameter: pipelinechannel--list-dict-maker-1-2-Output-loop-item + parameterExpressionSelector: parseJson(string_value)["a"] + taskInfo: + name: print-int-6 + inputDefinitions: + parameters: + pipelinechannel--list-dict-maker-1-2-Output: + parameterType: LIST + pipelinechannel--list-dict-maker-1-2-Output-loop-item: + parameterType: STRING comp-for-loop-2: dag: tasks: @@ -129,6 +213,112 @@ components: parameters: pipelinechannel--loop-item-param-5: parameterType: STRUCT + comp-for-loop-8: + dag: + tasks: + print-int: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-int + inputs: + parameters: + x: + componentInputParameter: pipelinechannel--loop-item-param-7 + parameterExpressionSelector: parseJson(string_value)["a"] + taskInfo: + name: print-int + inputDefinitions: + parameters: + pipelinechannel--loop-item-param-7: + parameterType: STRUCT + comp-for-loop-9: + dag: + tasks: + print-int-2: + cachingOptions: + enableCache: true + componentRef: + name: comp-print-int-2 + inputs: + parameters: + x: + componentInputParameter: pipelinechannel--list-dict-maker-0-Output-loop-item + parameterExpressionSelector: parseJson(string_value)["a"] + taskInfo: + name: print-int-2 + inputDefinitions: + parameters: + pipelinechannel--list-dict-maker-0-Output: + parameterType: LIST + pipelinechannel--list-dict-maker-0-Output-loop-item: + parameterType: STRUCT + comp-list-dict-maker-0: + executorLabel: exec-list-dict-maker-0 + outputDefinitions: + parameters: + Output: + parameterType: LIST + comp-list-dict-maker-1: + executorLabel: exec-list-dict-maker-1 + outputDefinitions: + parameters: + Output: + parameterType: LIST + comp-list-dict-maker-1-2: + executorLabel: exec-list-dict-maker-1-2 + outputDefinitions: + parameters: + Output: + parameterType: LIST + comp-list-dict-maker-2: + executorLabel: exec-list-dict-maker-2 + outputDefinitions: + parameters: + Output: + parameterType: LIST + comp-list-dict-maker-3: + executorLabel: exec-list-dict-maker-3 + outputDefinitions: + parameters: + Output: + parameterType: LIST + comp-print-int: + executorLabel: exec-print-int + inputDefinitions: + parameters: + x: + parameterType: NUMBER_INTEGER + comp-print-int-2: + executorLabel: exec-print-int-2 + inputDefinitions: + parameters: + x: + parameterType: NUMBER_INTEGER + comp-print-int-3: + executorLabel: exec-print-int-3 + inputDefinitions: + parameters: + x: + parameterType: NUMBER_INTEGER + comp-print-int-4: + executorLabel: exec-print-int-4 + inputDefinitions: + parameters: + x: + parameterType: NUMBER_INTEGER + comp-print-int-5: + executorLabel: exec-print-int-5 + inputDefinitions: + parameters: + x: + parameterType: NUMBER_INTEGER + comp-print-int-6: + executorLabel: exec-print-int-6 + inputDefinitions: + parameters: + x: + parameterType: NUMBER_INTEGER comp-print-text: executorLabel: exec-print-text inputDefinitions: @@ -167,6 +357,330 @@ components: parameterType: STRING deploymentSpec: executors: + exec-list-dict-maker-0: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - list_dict_maker_0 + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef list_dict_maker_0() -> List[Dict[str, int]]:\n \"\"\"Enforces\ + \ strict type checking - returns a list of dictionaries \n where keys\ + \ are strings and values are integers. For testing type \n handling during\ + \ compilation.\"\"\"\n return [{'a': 1, 'b': 2}, {'a': 2, 'b': 3}, {'a':\ + \ 3, 'b': 4}]\n\n" + image: python:3.7 + exec-list-dict-maker-1: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - list_dict_maker_1 + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef list_dict_maker_1() -> List[Dict]:\n \"\"\"Utilizes generic\ + \ dictionary typing (no enforcement of specific key or\n value types).\n\ + \n Tests flexibility in type handling.\n \"\"\"\n return [{'a':\ + \ 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}]\n\n" + image: python:3.7 + exec-list-dict-maker-1-2: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - list_dict_maker_1 + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef list_dict_maker_1() -> List[Dict]:\n \"\"\"Utilizes generic\ + \ dictionary typing (no enforcement of specific key or\n value types).\n\ + \n Tests flexibility in type handling.\n \"\"\"\n return [{'a':\ + \ 1, 'b': 2}, {'a': 2, 'b': 3}, {'a': 3, 'b': 4}]\n\n" + image: python:3.7 + exec-list-dict-maker-2: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - list_dict_maker_2 + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef list_dict_maker_2() -> List[dict]:\n \"\"\"Returns a list\ + \ of dictionaries without type enforcement.\n\n Tests flexibility in\ + \ type handling.\n \"\"\"\n return [{'a': 1, 'b': 2}, {'a': 2, 'b':\ + \ 3}, {'a': 3, 'b': 4}]\n\n" + image: python:3.7 + exec-list-dict-maker-3: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - list_dict_maker_3 + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef list_dict_maker_3() -> List:\n \"\"\"Returns a basic list\ + \ (no typing or structure guarantees).\n\n Tests the limits of compiler\ + \ type handling.\n \"\"\"\n return [{'a': 1, 'b': 2}, {'a': 2, 'b':\ + \ 3}, {'a': 3, 'b': 4}]\n\n" + image: python:3.7 + exec-print-int: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_int + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_int(x: int):\n print(x)\n\n" + image: python:3.7 + exec-print-int-2: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_int + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_int(x: int):\n print(x)\n\n" + image: python:3.7 + exec-print-int-3: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_int + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_int(x: int):\n print(x)\n\n" + image: python:3.7 + exec-print-int-4: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_int + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_int(x: int):\n print(x)\n\n" + image: python:3.7 + exec-print-int-5: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_int + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_int(x: int):\n print(x)\n\n" + image: python:3.7 + exec-print-int-6: + container: + args: + - --executor_input + - '{{$}}' + - --function_to_execute + - print_int + command: + - sh + - -c + - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ + \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ + \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ + $0\" \"$@\"\n" + - sh + - -ec + - 'program_path=$(mktemp -d) + + + printf "%s" "$0" > "$program_path/ephemeral_component.py" + + _KFP_RUNTIME=true python3 -m kfp.dsl.executor_main --component_module_path "$program_path/ephemeral_component.py" "$@" + + ' + - "\nimport kfp\nfrom kfp import dsl\nfrom kfp.dsl import *\nfrom typing import\ + \ *\n\ndef print_int(x: int):\n print(x)\n\n" + image: python:3.7 exec-print-text: container: args: @@ -179,7 +693,7 @@ deploymentSpec: - -c - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ - \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.1.3'\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ $0\" \"$@\"\n" - sh @@ -207,7 +721,7 @@ deploymentSpec: - -c - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ - \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.1.3'\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ $0\" \"$@\"\n" - sh @@ -235,7 +749,7 @@ deploymentSpec: - -c - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ - \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.1.3'\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ $0\" \"$@\"\n" - sh @@ -263,7 +777,7 @@ deploymentSpec: - -c - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ - \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.1.3'\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ $0\" \"$@\"\n" - sh @@ -291,7 +805,7 @@ deploymentSpec: - -c - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ - \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.1.3'\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ $0\" \"$@\"\n" - sh @@ -319,7 +833,7 @@ deploymentSpec: - -c - "\nif ! [ -x \"$(command -v pip)\" ]; then\n python3 -m ensurepip ||\ \ python3 -m ensurepip --user || apt-get install python3-pip\nfi\n\nPIP_DISABLE_PIP_VERSION_CHECK=1\ - \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.1.3'\ + \ python3 -m pip install --quiet --no-warn-script-location 'kfp==2.6.0'\ \ '--no-deps' 'typing-extensions>=3.7.4,<5; python_version<\"3.9\"' && \"\ $0\" \"$@\"\n" - sh @@ -355,6 +869,74 @@ root: inputParameter: pipelinechannel--loop_parameter taskInfo: name: for-loop-1 + for-loop-10: + componentRef: + name: comp-for-loop-10 + dependentTasks: + - list-dict-maker-1 + inputs: + parameters: + pipelinechannel--list-dict-maker-1-Output: + taskOutputParameter: + outputParameterKey: Output + producerTask: list-dict-maker-1 + parameterIterator: + itemInput: pipelinechannel--list-dict-maker-1-Output-loop-item + items: + inputParameter: pipelinechannel--list-dict-maker-1-Output + taskInfo: + name: for-loop-10 + for-loop-11: + componentRef: + name: comp-for-loop-11 + dependentTasks: + - list-dict-maker-2 + inputs: + parameters: + pipelinechannel--list-dict-maker-2-Output: + taskOutputParameter: + outputParameterKey: Output + producerTask: list-dict-maker-2 + parameterIterator: + itemInput: pipelinechannel--list-dict-maker-2-Output-loop-item + items: + inputParameter: pipelinechannel--list-dict-maker-2-Output + taskInfo: + name: for-loop-11 + for-loop-12: + componentRef: + name: comp-for-loop-12 + dependentTasks: + - list-dict-maker-3 + inputs: + parameters: + pipelinechannel--list-dict-maker-3-Output: + taskOutputParameter: + outputParameterKey: Output + producerTask: list-dict-maker-3 + parameterIterator: + itemInput: pipelinechannel--list-dict-maker-3-Output-loop-item + items: + inputParameter: pipelinechannel--list-dict-maker-3-Output + taskInfo: + name: for-loop-12 + for-loop-13: + componentRef: + name: comp-for-loop-13 + dependentTasks: + - list-dict-maker-1-2 + inputs: + parameters: + pipelinechannel--list-dict-maker-1-2-Output: + taskOutputParameter: + outputParameterKey: Output + producerTask: list-dict-maker-1-2 + parameterIterator: + itemInput: pipelinechannel--list-dict-maker-1-2-Output-loop-item + items: + inputParameter: pipelinechannel--list-dict-maker-1-2-Output + taskInfo: + name: for-loop-13 for-loop-4: componentRef: name: comp-for-loop-4 @@ -364,9 +946,72 @@ root: raw: '[{"A_a": "1", "B_b": "2"}, {"A_a": "10", "B_b": "20"}]' taskInfo: name: for-loop-4 + for-loop-8: + componentRef: + name: comp-for-loop-8 + iteratorPolicy: + parallelismLimit: 1 + parameterIterator: + itemInput: pipelinechannel--loop-item-param-7 + items: + raw: '[{"a": 1, "b": 2}, {"a": 2, "b": 3}, {"a": 3, "b": 4}]' + taskInfo: + name: for-loop-8 + for-loop-9: + componentRef: + name: comp-for-loop-9 + dependentTasks: + - list-dict-maker-0 + inputs: + parameters: + pipelinechannel--list-dict-maker-0-Output: + taskOutputParameter: + outputParameterKey: Output + producerTask: list-dict-maker-0 + parameterIterator: + itemInput: pipelinechannel--list-dict-maker-0-Output-loop-item + items: + inputParameter: pipelinechannel--list-dict-maker-0-Output + taskInfo: + name: for-loop-9 + list-dict-maker-0: + cachingOptions: + enableCache: true + componentRef: + name: comp-list-dict-maker-0 + taskInfo: + name: list-dict-maker-0 + list-dict-maker-1: + cachingOptions: + enableCache: true + componentRef: + name: comp-list-dict-maker-1 + taskInfo: + name: list-dict-maker-1 + list-dict-maker-1-2: + cachingOptions: + enableCache: true + componentRef: + name: comp-list-dict-maker-1-2 + taskInfo: + name: list-dict-maker-1-2 + list-dict-maker-2: + cachingOptions: + enableCache: true + componentRef: + name: comp-list-dict-maker-2 + taskInfo: + name: list-dict-maker-2 + list-dict-maker-3: + cachingOptions: + enableCache: true + componentRef: + name: comp-list-dict-maker-3 + taskInfo: + name: list-dict-maker-3 inputDefinitions: parameters: loop_parameter: parameterType: LIST schemaVersion: 2.1.0 -sdkVersion: kfp-2.1.3 +sdkVersion: kfp-2.6.0 diff --git a/sdk/python/test_data/test_data_config.yaml b/sdk/python/test_data/test_data_config.yaml index dc8b23ba6c..ddfa0802f5 100644 --- a/sdk/python/test_data/test_data_config.yaml +++ b/sdk/python/test_data/test_data_config.yaml @@ -192,6 +192,9 @@ pipelines: - module: cross_loop_after_topology name: my_pipeline execute: false + - module: pipeline_with_parallelfor_list_artifacts + name: my_pipeline + execute: false components: test_data_dir: sdk/python/test_data/components read: true