feat(sdk): Support pipeline outputs (#8204)

* Support pipeline outputs

* release note
This commit is contained in:
Chen Sun 2022-08-29 13:26:53 -07:00 committed by GitHub
parent 72c1d10a33
commit 48574dc648
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 455 additions and 117 deletions

View File

@ -3,6 +3,7 @@
## Major Features and Improvements
* Support parallelism setting in ParallelFor [\#8146](https://github.com/kubeflow/pipelines/pull/8146)
* Support for Python v3.10 [\#8186](https://github.com/kubeflow/pipelines/pull/8186)
* Support pipeline as a component [\#8179](https://github.com/kubeflow/pipelines/pull/8179), [\#8204](https://github.com/kubeflow/pipelines/pull/8204)
## Breaking Changes

View File

@ -49,6 +49,7 @@ CONFIG = {
'pipeline_with_parallelfor_parallelism',
'pipeline_in_pipeline',
'pipeline_in_pipeline_complex',
'pipeline_with_outputs',
],
'test_data_dir': 'sdk/python/kfp/compiler/test_data/pipelines',
'config': {

View File

@ -73,11 +73,25 @@ class Compiler:
with type_utils.TypeCheckManager(enable=type_check):
if isinstance(pipeline_func, graph_component.GraphComponent):
pipeline_spec = self._create_pipeline(
pipeline_func=pipeline_func.pipeline_func,
pipeline_name=pipeline_name,
pipeline_parameters_override=pipeline_parameters,
)
# Retrieve the pre-comppiled pipeline spec.
pipeline_spec = pipeline_func.component_spec.implementation.graph
# Verify that pipeline_parameters contains only input names
# that match the pipeline inputs definition.
for input_name, input_value in (pipeline_parameters or
{}).items():
if input_name in pipeline_spec.root.input_definitions.parameters:
pipeline_spec.root.input_definitions.parameters[
input_name].default_value.CopyFrom(
builder.to_protobuf_value(input_value))
elif input_name in pipeline_spec.root.input_definitions.artifacts:
raise NotImplementedError(
'Default value for artifact input is not supported yet.'
)
else:
raise ValueError(
'Pipeline parameter {} does not match any known '
'pipeline input.'.format(input_name))
elif isinstance(pipeline_func, base_component.BaseComponent):
component_spec = builder.modify_component_spec_for_compile(
@ -94,87 +108,3 @@ class Compiler:
f'decorator. Got: {type(pipeline_func)}')
builder.write_pipeline_spec_to_file(
pipeline_spec=pipeline_spec, package_path=package_path)
def _create_pipeline(
self,
pipeline_func: Callable[..., Any],
pipeline_name: Optional[str] = None,
pipeline_parameters_override: Optional[Mapping[str, Any]] = None,
) -> pipeline_spec_pb2.PipelineSpec:
"""Creates a pipeline instance and constructs the pipeline spec from
it.
Args:
pipeline_func: The pipeline function with @dsl.pipeline decorator.
pipeline_name: Optional; the name of the pipeline.
pipeline_parameters_override: Optional; the mapping from parameter
names to values.
Returns:
A PipelineSpec proto representing the compiled pipeline.
"""
# pipeline_func is a GraphComponent instance, retrieve its the original
# pipeline function
pipeline_func = getattr(pipeline_func, 'python_func', pipeline_func)
# Create the arg list with no default values and call pipeline function.
# Assign type information to the PipelineChannel
pipeline_meta = component_factory.extract_component_interface(
pipeline_func)
pipeline_name = pipeline_name or pipeline_meta.name
pipeline_root = getattr(pipeline_func, 'pipeline_root', None)
args_list = []
signature = inspect.signature(pipeline_func)
for arg_name in signature.parameters:
arg_type = pipeline_meta.inputs[arg_name].type
args_list.append(
pipeline_channel.create_pipeline_channel(
name=arg_name,
channel_type=arg_type,
))
with pipeline_context.Pipeline(pipeline_name) as dsl_pipeline:
pipeline_func(*args_list)
if not dsl_pipeline.tasks:
raise ValueError('Task is missing from pipeline.')
pipeline_inputs = pipeline_meta.inputs or {}
# Verify that pipeline_parameters_override contains only input names
# that match the pipeline inputs definition.
pipeline_parameters_override = pipeline_parameters_override or {}
for input_name in pipeline_parameters_override:
if input_name not in pipeline_inputs:
raise ValueError(
'Pipeline parameter {} does not match any known '
'pipeline argument.'.format(input_name))
# Fill in the default values.
args_list_with_defaults = [
pipeline_channel.create_pipeline_channel(
name=input_name,
channel_type=input_spec.type,
value=pipeline_parameters_override.get(input_name) or
input_spec.default,
) for input_name, input_spec in pipeline_inputs.items()
]
# Making the pipeline group name unique to prevent name clashes with
# templates
pipeline_group = dsl_pipeline.groups[0]
pipeline_group.name = uuid.uuid4().hex
pipeline_spec, _ = builder.create_pipeline_spec_and_deployment_config(
pipeline_args=args_list_with_defaults,
pipeline=dsl_pipeline,
)
if pipeline_root:
pipeline_spec.default_pipeline_root = pipeline_root
return pipeline_spec

View File

@ -12,12 +12,13 @@
# See the License for the specific language governing permissions and
# limitations under the License.
import collections
import json
import os
import re
import subprocess
import tempfile
from typing import Any, Dict, List, Optional
from typing import Any, Dict, List, NamedTuple, Optional
import unittest
from absl.testing import parameterized
@ -681,6 +682,48 @@ implementation:
self.assertTrue('exec-print-op' in
pipeline_spec['deploymentSpec']['executors'])
def test_pipeline_with_invalid_output(self):
with self.assertRaisesRegex(ValueError,
'Pipeline output not defined: msg1'):
@dsl.component
def print_op(msg: str) -> str:
print(msg)
@dsl.pipeline
def my_pipeline() -> NamedTuple('Outputs', [
('msg', str),
]):
task = print_op(msg='Hello')
output = collections.namedtuple('Outputs', ['msg1'])
return output(task.output)
def test_pipeline_with_missing_output(self):
with self.assertRaisesRegex(ValueError, 'Missing pipeline output: msg'):
@dsl.component
def print_op(msg: str) -> str:
print(msg)
@dsl.pipeline
def my_pipeline() -> NamedTuple('Outputs', [
('msg', str),
]):
task = print_op(msg='Hello')
with self.assertRaisesRegex(ValueError,
'Missing pipeline output: model'):
@dsl.component
def print_op(msg: str) -> str:
print(msg)
@dsl.pipeline
def my_pipeline() -> NamedTuple('Outputs', [
('model', dsl.Model),
]):
task = print_op(msg='Hello')
class V2NamespaceAliasTest(unittest.TestCase):
"""Test that imports of both modules and objects are aliased (e.g. all

View File

@ -42,6 +42,8 @@ group_type_to_dsl_class = {
tasks_group.TasksGroupType.EXIT_HANDLER: tasks_group.ExitHandler,
}
_SINGLE_OUTPUT_NAME = 'Output'
def _additional_input_name_for_pipeline_channel(
channel_or_name: Union[pipeline_channel.PipelineChannel, str]) -> str:
@ -55,7 +57,7 @@ def _additional_input_name_for_pipeline_channel(
channel_or_name)
def _to_protobuf_value(value: type_utils.PARAMETER_TYPES) -> struct_pb2.Value:
def to_protobuf_value(value: type_utils.PARAMETER_TYPES) -> struct_pb2.Value:
"""Creates a google.protobuf.struct_pb2.Value message out of a provide
value.
@ -77,11 +79,11 @@ def _to_protobuf_value(value: type_utils.PARAMETER_TYPES) -> struct_pb2.Value:
elif isinstance(value, dict):
return struct_pb2.Value(
struct_value=struct_pb2.Struct(
fields={k: _to_protobuf_value(v) for k, v in value.items()}))
fields={k: to_protobuf_value(v) for k, v in value.items()}))
elif isinstance(value, list):
return struct_pb2.Value(
list_value=struct_pb2.ListValue(
values=[_to_protobuf_value(v) for v in value]))
values=[to_protobuf_value(v) for v in value]))
else:
raise ValueError('Value must be one of the following types: '
'str, int, float, bool, dict, and list. Got: '
@ -298,7 +300,7 @@ def build_task_spec_for_task(
pipeline_task_spec.inputs.parameters[
input_name].runtime_value.constant.CopyFrom(
_to_protobuf_value(input_value))
to_protobuf_value(input_value))
else:
raise ValueError(
@ -360,7 +362,7 @@ def build_component_spec_for_task(
if input_spec.default is not None:
component_spec.input_definitions.parameters[
input_name].default_value.CopyFrom(
_to_protobuf_value(input_spec.default))
to_protobuf_value(input_spec.default))
else:
component_spec.input_definitions.artifacts[
@ -380,6 +382,107 @@ def build_component_spec_for_task(
return component_spec
# TODO(chensun): merge with build_component_spec_for_task
def _build_component_spec_from_component_spec_structure(
component_spec_struct: structures.ComponentSpec,
) -> pipeline_spec_pb2.ComponentSpec:
"""Builds ComponentSpec proto from ComponentSpec structure."""
component_spec = pipeline_spec_pb2.ComponentSpec()
for input_name, input_spec in (component_spec_struct.inputs or {}).items():
# Special handling for PipelineTaskFinalStatus first.
if type_utils.is_task_final_status_type(input_spec.type):
component_spec.input_definitions.parameters[
input_name].parameter_type = pipeline_spec_pb2.ParameterType.STRUCT
continue
if type_utils.is_parameter_type(input_spec.type):
component_spec.input_definitions.parameters[
input_name].parameter_type = type_utils.get_parameter_type(
input_spec.type)
if input_spec.default is not None:
component_spec.input_definitions.parameters[
input_name].default_value.CopyFrom(
to_protobuf_value(input_spec.default))
else:
component_spec.input_definitions.artifacts[
input_name].artifact_type.CopyFrom(
type_utils.get_artifact_type_schema(input_spec.type))
for output_name, output_spec in (component_spec_struct.outputs or
{}).items():
if type_utils.is_parameter_type(output_spec.type):
component_spec.output_definitions.parameters[
output_name].parameter_type = type_utils.get_parameter_type(
output_spec.type)
else:
component_spec.output_definitions.artifacts[
output_name].artifact_type.CopyFrom(
type_utils.get_artifact_type_schema(output_spec.type))
return component_spec
def _connect_dag_outputs(
component_spec: pipeline_spec_pb2.ComponentSpec,
output_name: str,
output_channel: pipeline_channel.PipelineChannel,
) -> None:
"""Connects dag ouptut to a subtask output.
Args:
component_spec: The component spec to modify its dag outputs.
output_name: The name of the dag output.
output_channel: The pipeline channel selected for the dag output.
"""
if isinstance(output_channel, pipeline_channel.PipelineArtifactChannel):
if output_name not in component_spec.output_definitions.artifacts:
raise ValueError(f'Pipeline output not defined: {output_name}.')
component_spec.dag.outputs.artifacts[
output_name].artifact_selectors.append(
pipeline_spec_pb2.DagOutputsSpec.ArtifactSelectorSpec(
producer_subtask=output_channel.task_name,
output_artifact_key=output_channel.name,
))
elif isinstance(output_channel, pipeline_channel.PipelineParameterChannel):
if output_name not in component_spec.output_definitions.parameters:
raise ValueError(f'Pipeline output not defined: {output_name}.')
component_spec.dag.outputs.parameters[
output_name].value_from_parameter.producer_subtask = output_channel.task_name
component_spec.dag.outputs.parameters[
output_name].value_from_parameter.output_parameter_key = output_channel.name
def _build_dag_outputs(
component_spec: pipeline_spec_pb2.ComponentSpec,
dag_outputs: Optional[Any],
) -> None:
"""Builds DAG output spec."""
if dag_outputs is not None:
if isinstance(dag_outputs, pipeline_channel.PipelineChannel):
_connect_dag_outputs(
component_spec=component_spec,
output_name=_SINGLE_OUTPUT_NAME,
output_channel=dag_outputs,
)
elif isinstance(dag_outputs, tuple) and hasattr(dag_outputs, '_asdict'):
for output_name, output_channel in dag_outputs._asdict().items():
_connect_dag_outputs(
component_spec=component_spec,
output_name=output_name,
output_channel=output_channel,
)
# Valid dag outputs covers all outptus in component definition.
for output_name in component_spec.output_definitions.artifacts:
if output_name not in component_spec.dag.outputs.artifacts:
raise ValueError(f'Missing pipeline output: {output_name}.')
for output_name in component_spec.output_definitions.parameters:
if output_name not in component_spec.dag.outputs.parameters:
raise ValueError(f'Missing pipeline output: {output_name}.')
def build_importer_spec_for_task(
task: pipeline_task.PipelineTask
) -> pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec:
@ -480,11 +583,11 @@ def _fill_in_component_input_default_value(
elif pipeline_spec_pb2.ParameterType.STRUCT == parameter_type:
component_spec.input_definitions.parameters[
input_name].default_value.CopyFrom(
_to_protobuf_value(default_value))
to_protobuf_value(default_value))
elif pipeline_spec_pb2.ParameterType.LIST == parameter_type:
component_spec.input_definitions.parameters[
input_name].default_value.CopyFrom(
_to_protobuf_value(default_value))
to_protobuf_value(default_value))
def build_component_spec_for_group(
@ -1369,14 +1472,16 @@ def _merge_component_spec(
def create_pipeline_spec_and_deployment_config(
pipeline: pipeline_context.Pipeline,
pipeline_args: List[pipeline_channel.PipelineChannel],
component_spec: structures.ComponentSpec,
pipeline_outputs: Optional[Any] = None,
) -> Tuple[pipeline_spec_pb2.PipelineSpec,
pipeline_spec_pb2.PipelineDeploymentConfig]:
"""Creates a pipeline spec object.
Args:
pipeline: The instantiated pipeline object.
pipeline_args: The list of pipeline input parameters.
component_spec: The component spec structures.
pipeline_outputs: The pipeline outputs via return.
Returns:
A tuple of PipelineSpec proto representing the compiled pipeline and its
@ -1396,10 +1501,10 @@ def create_pipeline_spec_and_deployment_config(
pipeline_spec.schema_version = '2.1.0'
pipeline_spec.root.CopyFrom(
build_component_spec_for_group(
pipeline_channels=pipeline_args,
is_root_group=True,
))
_build_component_spec_from_component_spec_structure(component_spec))
_build_dag_outputs(
component_spec=pipeline_spec.root, dag_outputs=pipeline_outputs)
root_group = pipeline.groups[0]

View File

@ -0,0 +1,65 @@
# Copyright 2022 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.
import collections
from typing import NamedTuple
from kfp import compiler
from kfp import dsl
from kfp.dsl import Artifact
from kfp.dsl import Output
@dsl.component
def print_op1(msg: str) -> str:
print(msg)
return msg
@dsl.container_component
def print_op2(msg: str, data: Output[Artifact]):
return dsl.ContainerSpec(
image='alpine',
command=[
'sh',
'-c',
'mkdir --parents $(dirname "$1") && echo "$0" > "$1"',
],
args=[msg, data.path],
)
@dsl.pipeline
def inner_pipeline(
msg: str) -> NamedTuple('Outputs', [
('msg', str),
('data', Artifact),
]):
task1 = print_op1(msg=msg)
task2 = print_op2(msg=task1.output)
output = collections.namedtuple('Outputs', ['msg', 'data'])
return output(task1.output, task2.output)
@dsl.pipeline(name='pipeline-in-pipeline')
def my_pipeline() -> Artifact:
task1 = print_op1(msg='Hello')
task2 = inner_pipeline(msg='world')
return task2.outputs['data']
if __name__ == '__main__':
compiler.Compiler().compile(
pipeline_func=my_pipeline,
package_path=__file__.replace('.py', '.yaml'))

View File

@ -0,0 +1,195 @@
components:
comp-inner-pipeline:
dag:
outputs:
artifacts:
data:
artifactSelectors:
- outputArtifactKey: data
producerSubtask: print-op2
parameters:
msg:
valueFromParameter:
outputParameterKey: Output
producerSubtask: print-op1
tasks:
print-op1:
cachingOptions:
enableCache: true
componentRef:
name: comp-print-op1-2
inputs:
parameters:
msg:
componentInputParameter: msg
taskInfo:
name: print-op1
print-op2:
cachingOptions:
enableCache: true
componentRef:
name: comp-print-op2
dependentTasks:
- print-op1
inputs:
parameters:
msg:
taskOutputParameter:
outputParameterKey: Output
producerTask: print-op1
taskInfo:
name: print-op2
inputDefinitions:
parameters:
msg:
parameterType: STRING
outputDefinitions:
artifacts:
data:
artifactType:
schemaTitle: system.Artifact
schemaVersion: 0.0.1
parameters:
msg:
parameterType: STRING
comp-print-op1:
executorLabel: exec-print-op1
inputDefinitions:
parameters:
msg:
parameterType: STRING
outputDefinitions:
parameters:
Output:
parameterType: STRING
comp-print-op1-2:
executorLabel: exec-print-op1-2
inputDefinitions:
parameters:
msg:
parameterType: STRING
outputDefinitions:
parameters:
Output:
parameterType: STRING
comp-print-op2:
executorLabel: exec-print-op2
inputDefinitions:
parameters:
msg:
parameterType: STRING
outputDefinitions:
artifacts:
data:
artifactType:
schemaTitle: system.Artifact
schemaVersion: 0.0.1
deploymentSpec:
executors:
exec-print-op1:
container:
args:
- --executor_input
- '{{$}}'
- --function_to_execute
- print_op1
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.0.0-beta.3'\
\ && \"$0\" \"$@\"\n"
- sh
- -ec
- 'program_path=$(mktemp -d)
printf "%s" "$0" > "$program_path/ephemeral_component.py"
python3 -m kfp.components.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_op1(msg: str) -> str:\n print(msg)\n return msg\n\
\n"
image: python:3.7
exec-print-op1-2:
container:
args:
- --executor_input
- '{{$}}'
- --function_to_execute
- print_op1
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.0.0-beta.3'\
\ && \"$0\" \"$@\"\n"
- sh
- -ec
- 'program_path=$(mktemp -d)
printf "%s" "$0" > "$program_path/ephemeral_component.py"
python3 -m kfp.components.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_op1(msg: str) -> str:\n print(msg)\n return msg\n\
\n"
image: python:3.7
exec-print-op2:
container:
args:
- '{{$.inputs.parameters[''msg'']}}'
- '{{$.outputs.artifacts[''data''].path}}'
command:
- sh
- -c
- mkdir --parents $(dirname "$1") && echo "$0" > "$1"
image: alpine
pipelineInfo:
name: pipeline-in-pipeline
root:
dag:
outputs:
artifacts:
Output:
artifactSelectors:
- outputArtifactKey: data
producerSubtask: inner-pipeline
tasks:
inner-pipeline:
cachingOptions:
enableCache: true
componentRef:
name: comp-inner-pipeline
inputs:
parameters:
msg:
runtimeValue:
constant: world
taskInfo:
name: inner-pipeline
print-op1:
cachingOptions:
enableCache: true
componentRef:
name: comp-print-op1
inputs:
parameters:
msg:
runtimeValue:
constant: Hello
taskInfo:
name: print-op1
outputDefinitions:
artifacts:
Output:
artifactType:
schemaTitle: system.Artifact
schemaVersion: 0.0.1
schemaVersion: 2.1.0
sdkVersion: kfp-2.0.0-beta.3

View File

@ -54,22 +54,11 @@ class GraphComponent(base_component.BaseComponent):
with pipeline_context.Pipeline(
self.component_spec.name) as dsl_pipeline:
pipeline_func(*args_list)
pipeline_outputs = pipeline_func(*args_list)
if not dsl_pipeline.tasks:
raise ValueError('Task is missing from pipeline.')
component_inputs = self.component_spec.inputs or {}
# Fill in the default values.
args_list_with_defaults = [
pipeline_channel.create_pipeline_channel(
name=input_name,
channel_type=input_spec.type,
value=input_spec.default,
) for input_name, input_spec in component_inputs.items()
]
# Making the pipeline group name unique to prevent name clashes with
# templates
pipeline_group = dsl_pipeline.groups[0]
@ -77,7 +66,14 @@ class GraphComponent(base_component.BaseComponent):
self.pipeline_spec, self.deployment_config = (
builder.create_pipeline_spec_and_deployment_config(
pipeline=dsl_pipeline, pipeline_args=args_list_with_defaults))
pipeline=dsl_pipeline,
component_spec=self.component_spec,
pipeline_outputs=pipeline_outputs,
))
pipeline_root = getattr(pipeline_func, 'pipeline_root', None)
if pipeline_root is not None:
self.pipeline_spec.default_pipeline_root = pipeline_root
self.component_spec.implementation.graph = self.pipeline_spec

View File

@ -189,7 +189,9 @@ class PipelineTask:
Used when a task has exactly one output parameter.
"""
if len(self._outputs) != 1:
raise AttributeError
raise AttributeError(
'The task has multiple outputs. Please reference the output by its name.'
)
return list(self._outputs.values())[0]
@property