feat(sdk): Support pipeline outputs (#8204)
* Support pipeline outputs * release note
This commit is contained in:
parent
72c1d10a33
commit
48574dc648
|
@ -3,6 +3,7 @@
|
||||||
## Major Features and Improvements
|
## Major Features and Improvements
|
||||||
* Support parallelism setting in ParallelFor [\#8146](https://github.com/kubeflow/pipelines/pull/8146)
|
* 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 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
|
## Breaking Changes
|
||||||
|
|
||||||
|
|
|
@ -49,6 +49,7 @@ CONFIG = {
|
||||||
'pipeline_with_parallelfor_parallelism',
|
'pipeline_with_parallelfor_parallelism',
|
||||||
'pipeline_in_pipeline',
|
'pipeline_in_pipeline',
|
||||||
'pipeline_in_pipeline_complex',
|
'pipeline_in_pipeline_complex',
|
||||||
|
'pipeline_with_outputs',
|
||||||
],
|
],
|
||||||
'test_data_dir': 'sdk/python/kfp/compiler/test_data/pipelines',
|
'test_data_dir': 'sdk/python/kfp/compiler/test_data/pipelines',
|
||||||
'config': {
|
'config': {
|
||||||
|
|
|
@ -73,11 +73,25 @@ class Compiler:
|
||||||
|
|
||||||
with type_utils.TypeCheckManager(enable=type_check):
|
with type_utils.TypeCheckManager(enable=type_check):
|
||||||
if isinstance(pipeline_func, graph_component.GraphComponent):
|
if isinstance(pipeline_func, graph_component.GraphComponent):
|
||||||
pipeline_spec = self._create_pipeline(
|
# Retrieve the pre-comppiled pipeline spec.
|
||||||
pipeline_func=pipeline_func.pipeline_func,
|
pipeline_spec = pipeline_func.component_spec.implementation.graph
|
||||||
pipeline_name=pipeline_name,
|
|
||||||
pipeline_parameters_override=pipeline_parameters,
|
# 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):
|
elif isinstance(pipeline_func, base_component.BaseComponent):
|
||||||
component_spec = builder.modify_component_spec_for_compile(
|
component_spec = builder.modify_component_spec_for_compile(
|
||||||
|
@ -94,87 +108,3 @@ class Compiler:
|
||||||
f'decorator. Got: {type(pipeline_func)}')
|
f'decorator. Got: {type(pipeline_func)}')
|
||||||
builder.write_pipeline_spec_to_file(
|
builder.write_pipeline_spec_to_file(
|
||||||
pipeline_spec=pipeline_spec, package_path=package_path)
|
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
|
|
||||||
|
|
|
@ -12,12 +12,13 @@
|
||||||
# See the License for the specific language governing permissions and
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
|
|
||||||
|
import collections
|
||||||
import json
|
import json
|
||||||
import os
|
import os
|
||||||
import re
|
import re
|
||||||
import subprocess
|
import subprocess
|
||||||
import tempfile
|
import tempfile
|
||||||
from typing import Any, Dict, List, Optional
|
from typing import Any, Dict, List, NamedTuple, Optional
|
||||||
import unittest
|
import unittest
|
||||||
|
|
||||||
from absl.testing import parameterized
|
from absl.testing import parameterized
|
||||||
|
@ -681,6 +682,48 @@ implementation:
|
||||||
self.assertTrue('exec-print-op' in
|
self.assertTrue('exec-print-op' in
|
||||||
pipeline_spec['deploymentSpec']['executors'])
|
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):
|
class V2NamespaceAliasTest(unittest.TestCase):
|
||||||
"""Test that imports of both modules and objects are aliased (e.g. all
|
"""Test that imports of both modules and objects are aliased (e.g. all
|
||||||
|
|
|
@ -42,6 +42,8 @@ group_type_to_dsl_class = {
|
||||||
tasks_group.TasksGroupType.EXIT_HANDLER: tasks_group.ExitHandler,
|
tasks_group.TasksGroupType.EXIT_HANDLER: tasks_group.ExitHandler,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
_SINGLE_OUTPUT_NAME = 'Output'
|
||||||
|
|
||||||
|
|
||||||
def _additional_input_name_for_pipeline_channel(
|
def _additional_input_name_for_pipeline_channel(
|
||||||
channel_or_name: Union[pipeline_channel.PipelineChannel, str]) -> str:
|
channel_or_name: Union[pipeline_channel.PipelineChannel, str]) -> str:
|
||||||
|
@ -55,7 +57,7 @@ def _additional_input_name_for_pipeline_channel(
|
||||||
channel_or_name)
|
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
|
"""Creates a google.protobuf.struct_pb2.Value message out of a provide
|
||||||
value.
|
value.
|
||||||
|
|
||||||
|
@ -77,11 +79,11 @@ def _to_protobuf_value(value: type_utils.PARAMETER_TYPES) -> struct_pb2.Value:
|
||||||
elif isinstance(value, dict):
|
elif isinstance(value, dict):
|
||||||
return struct_pb2.Value(
|
return struct_pb2.Value(
|
||||||
struct_value=struct_pb2.Struct(
|
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):
|
elif isinstance(value, list):
|
||||||
return struct_pb2.Value(
|
return struct_pb2.Value(
|
||||||
list_value=struct_pb2.ListValue(
|
list_value=struct_pb2.ListValue(
|
||||||
values=[_to_protobuf_value(v) for v in value]))
|
values=[to_protobuf_value(v) for v in value]))
|
||||||
else:
|
else:
|
||||||
raise ValueError('Value must be one of the following types: '
|
raise ValueError('Value must be one of the following types: '
|
||||||
'str, int, float, bool, dict, and list. Got: '
|
'str, int, float, bool, dict, and list. Got: '
|
||||||
|
@ -298,7 +300,7 @@ def build_task_spec_for_task(
|
||||||
|
|
||||||
pipeline_task_spec.inputs.parameters[
|
pipeline_task_spec.inputs.parameters[
|
||||||
input_name].runtime_value.constant.CopyFrom(
|
input_name].runtime_value.constant.CopyFrom(
|
||||||
_to_protobuf_value(input_value))
|
to_protobuf_value(input_value))
|
||||||
|
|
||||||
else:
|
else:
|
||||||
raise ValueError(
|
raise ValueError(
|
||||||
|
@ -360,7 +362,7 @@ def build_component_spec_for_task(
|
||||||
if input_spec.default is not None:
|
if input_spec.default is not None:
|
||||||
component_spec.input_definitions.parameters[
|
component_spec.input_definitions.parameters[
|
||||||
input_name].default_value.CopyFrom(
|
input_name].default_value.CopyFrom(
|
||||||
_to_protobuf_value(input_spec.default))
|
to_protobuf_value(input_spec.default))
|
||||||
|
|
||||||
else:
|
else:
|
||||||
component_spec.input_definitions.artifacts[
|
component_spec.input_definitions.artifacts[
|
||||||
|
@ -380,6 +382,107 @@ def build_component_spec_for_task(
|
||||||
return component_spec
|
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(
|
def build_importer_spec_for_task(
|
||||||
task: pipeline_task.PipelineTask
|
task: pipeline_task.PipelineTask
|
||||||
) -> pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec:
|
) -> pipeline_spec_pb2.PipelineDeploymentConfig.ImporterSpec:
|
||||||
|
@ -480,11 +583,11 @@ def _fill_in_component_input_default_value(
|
||||||
elif pipeline_spec_pb2.ParameterType.STRUCT == parameter_type:
|
elif pipeline_spec_pb2.ParameterType.STRUCT == parameter_type:
|
||||||
component_spec.input_definitions.parameters[
|
component_spec.input_definitions.parameters[
|
||||||
input_name].default_value.CopyFrom(
|
input_name].default_value.CopyFrom(
|
||||||
_to_protobuf_value(default_value))
|
to_protobuf_value(default_value))
|
||||||
elif pipeline_spec_pb2.ParameterType.LIST == parameter_type:
|
elif pipeline_spec_pb2.ParameterType.LIST == parameter_type:
|
||||||
component_spec.input_definitions.parameters[
|
component_spec.input_definitions.parameters[
|
||||||
input_name].default_value.CopyFrom(
|
input_name].default_value.CopyFrom(
|
||||||
_to_protobuf_value(default_value))
|
to_protobuf_value(default_value))
|
||||||
|
|
||||||
|
|
||||||
def build_component_spec_for_group(
|
def build_component_spec_for_group(
|
||||||
|
@ -1369,14 +1472,16 @@ def _merge_component_spec(
|
||||||
|
|
||||||
def create_pipeline_spec_and_deployment_config(
|
def create_pipeline_spec_and_deployment_config(
|
||||||
pipeline: pipeline_context.Pipeline,
|
pipeline: pipeline_context.Pipeline,
|
||||||
pipeline_args: List[pipeline_channel.PipelineChannel],
|
component_spec: structures.ComponentSpec,
|
||||||
|
pipeline_outputs: Optional[Any] = None,
|
||||||
) -> Tuple[pipeline_spec_pb2.PipelineSpec,
|
) -> Tuple[pipeline_spec_pb2.PipelineSpec,
|
||||||
pipeline_spec_pb2.PipelineDeploymentConfig]:
|
pipeline_spec_pb2.PipelineDeploymentConfig]:
|
||||||
"""Creates a pipeline spec object.
|
"""Creates a pipeline spec object.
|
||||||
|
|
||||||
Args:
|
Args:
|
||||||
pipeline: The instantiated pipeline object.
|
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:
|
Returns:
|
||||||
A tuple of PipelineSpec proto representing the compiled pipeline and its
|
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.schema_version = '2.1.0'
|
||||||
|
|
||||||
pipeline_spec.root.CopyFrom(
|
pipeline_spec.root.CopyFrom(
|
||||||
build_component_spec_for_group(
|
_build_component_spec_from_component_spec_structure(component_spec))
|
||||||
pipeline_channels=pipeline_args,
|
|
||||||
is_root_group=True,
|
_build_dag_outputs(
|
||||||
))
|
component_spec=pipeline_spec.root, dag_outputs=pipeline_outputs)
|
||||||
|
|
||||||
root_group = pipeline.groups[0]
|
root_group = pipeline.groups[0]
|
||||||
|
|
||||||
|
|
|
@ -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'))
|
|
@ -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
|
|
@ -54,22 +54,11 @@ class GraphComponent(base_component.BaseComponent):
|
||||||
|
|
||||||
with pipeline_context.Pipeline(
|
with pipeline_context.Pipeline(
|
||||||
self.component_spec.name) as dsl_pipeline:
|
self.component_spec.name) as dsl_pipeline:
|
||||||
pipeline_func(*args_list)
|
pipeline_outputs = pipeline_func(*args_list)
|
||||||
|
|
||||||
if not dsl_pipeline.tasks:
|
if not dsl_pipeline.tasks:
|
||||||
raise ValueError('Task is missing from pipeline.')
|
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
|
# Making the pipeline group name unique to prevent name clashes with
|
||||||
# templates
|
# templates
|
||||||
pipeline_group = dsl_pipeline.groups[0]
|
pipeline_group = dsl_pipeline.groups[0]
|
||||||
|
@ -77,7 +66,14 @@ class GraphComponent(base_component.BaseComponent):
|
||||||
|
|
||||||
self.pipeline_spec, self.deployment_config = (
|
self.pipeline_spec, self.deployment_config = (
|
||||||
builder.create_pipeline_spec_and_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
|
self.component_spec.implementation.graph = self.pipeline_spec
|
||||||
|
|
||||||
|
|
|
@ -189,7 +189,9 @@ class PipelineTask:
|
||||||
Used when a task has exactly one output parameter.
|
Used when a task has exactly one output parameter.
|
||||||
"""
|
"""
|
||||||
if len(self._outputs) != 1:
|
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]
|
return list(self._outputs.values())[0]
|
||||||
|
|
||||||
@property
|
@property
|
||||||
|
|
Loading…
Reference in New Issue