WIP: KFP v2 POC (#5092)
* Changes: * add kfp v2 hello world sample * generate go proto code * code to unmarshal via jsonpb * generate container spec * add root DAG task and Makefile to run in argo * env setup to quickly build driver container * generate mlmd grpc golang client * mlmd grpc client usage example * driver-dag: put execution and context * driver dag: put context only when execution is a DAG * driver task: resolve parameter from parent execution * chain dag driver with task driver * driver: output argo parameters * driver: build driver docker image * driver: push image to dev registry * compiler: root dag driver compiler * mlmd doc * driver: use task spec and executor spec as driver input instead * driver: pod spec patch output for type EXECUTOR * compiler: passing - parameter placeholder with hello world example * include generated argo workflow in source control * driver no longer outputs parameters * publisher: publish parameters to execution custom properties * driver, publisher: execution state * publisher: remove built binary from repo * sample: add producer consumer sample * sample: producer consumer sample with only parameters * e2e output parameter support * e2e: driver resolves input parameter from tasks in the same DAG * compiler: convert KFP task dependency to argo task dependency * feat: refactor publisher so it can run as an entrypoint * build: entrypoint image * feat: executor and publisher in one container via entrypoint rewriting * fixed compile error (#1) * add licenses * update readme Co-authored-by: capri-xiyue <52932582+capri-xiyue@users.noreply.github.com>
This commit is contained in:
parent
c1bb01294a
commit
d024935339
|
|
@ -0,0 +1,7 @@
|
|||
PROTO_PATH=/usr/local/include/
|
||||
|
||||
gen_proto: v2alpha1/pipeline_spec.proto
|
||||
cd v2alpha1 && protoc -I=$(PROTO_PATH) -I=. \
|
||||
--go_out=go \
|
||||
--go_opt=paths=source_relative \
|
||||
pipeline_spec.proto
|
||||
|
|
@ -0,0 +1,19 @@
|
|||
# Pipeline Spec
|
||||
|
||||
## Generate golang proto code
|
||||
|
||||
Documentation: <https://developers.google.com/protocol-buffers/docs/reference/go-generated>
|
||||
|
||||
Download `protoc` compiler binary from: <https://github.com/protocolbuffers/protobuf/releases/tag/v3.14.0>.
|
||||
|
||||
Install proto code generator:
|
||||
|
||||
```bash
|
||||
go install google.golang.org/protobuf/cmd/protoc-gen-go
|
||||
```
|
||||
|
||||
Generate golang proto code:
|
||||
|
||||
```bash
|
||||
make gen_proto
|
||||
```
|
||||
File diff suppressed because it is too large
Load Diff
|
|
@ -1,5 +1,6 @@
|
|||
syntax = "proto3";
|
||||
|
||||
option go_package = "github.com/kubeflow/pipelines/api/v2alpha1/go";
|
||||
package ml_pipelines;
|
||||
|
||||
import "google/protobuf/any.proto";
|
||||
|
|
|
|||
|
|
@ -0,0 +1,97 @@
|
|||
# Copyright 2020 Google LLC
|
||||
#
|
||||
# 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.
|
||||
"""Pipeline DSL code for testing URI-based artifact passing."""
|
||||
from kfp import compiler
|
||||
from kfp import components
|
||||
from kfp import dsl
|
||||
|
||||
write_to_gcs = components.load_component_from_text(
|
||||
"""
|
||||
name: Write to GCS
|
||||
inputs:
|
||||
- {name: text, type: String, description: 'Content to be written to GCS'}
|
||||
outputs:
|
||||
- {name: output_gcs_path, type: String, description: 'GCS file path'}
|
||||
implementation:
|
||||
container:
|
||||
image: google/cloud-sdk:slim
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
echo "$0" | gsutil cp - "$1"
|
||||
- {inputValue: text}
|
||||
- {outputUri: output_gcs_path}
|
||||
"""
|
||||
)
|
||||
|
||||
read_from_gcs = components.load_component_from_text(
|
||||
"""
|
||||
name: Read from GCS
|
||||
inputs:
|
||||
- {name: input_gcs_path, type: String, description: 'GCS file path'}
|
||||
implementation:
|
||||
container:
|
||||
image: google/cloud-sdk:slim
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
gsutil cat "$0"
|
||||
- {inputUri: input_gcs_path}
|
||||
"""
|
||||
)
|
||||
|
||||
|
||||
def flip_coin_op():
|
||||
"""Flip a coin and output heads or tails randomly."""
|
||||
return dsl.ContainerOp(
|
||||
name='Flip coin',
|
||||
image='python:alpine3.6',
|
||||
command=['sh', '-c'],
|
||||
arguments=[
|
||||
'python -c "import random; result = \'heads\' if random.randint(0,1) == 0 '
|
||||
'else \'tails\'; print(result)" | tee /tmp/output'
|
||||
],
|
||||
file_outputs={'output': '/tmp/output'}
|
||||
)
|
||||
|
||||
|
||||
@dsl.pipeline(
|
||||
name='uri-artifact-pipeline',
|
||||
output_directory='gs://gongyuan-pipeline-test/v2-in-v1'
|
||||
)
|
||||
def uri_artifact(text='Hello world!'):
|
||||
task_1 = write_to_gcs(text=text)
|
||||
task_2 = read_from_gcs(input_gcs_path=task_1.outputs['output_gcs_path'])
|
||||
|
||||
# Test use URI within ParFor loop.
|
||||
loop_args = [1, 2, 3, 4]
|
||||
with dsl.ParallelFor(loop_args) as loop_arg:
|
||||
loop_task_2 = read_from_gcs(
|
||||
input_gcs_path=task_1.outputs['output_gcs_path']
|
||||
)
|
||||
|
||||
# Test use URI within condition.
|
||||
flip = flip_coin_op()
|
||||
with dsl.Condition(flip.output == 'heads'):
|
||||
condition_task_2 = read_from_gcs(
|
||||
input_gcs_path=task_1.outputs['output_gcs_path']
|
||||
)
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
compiler.Compiler().compile(uri_artifact, __file__ + '.yaml')
|
||||
|
|
@ -0,0 +1,168 @@
|
|||
apiVersion: argoproj.io/v1alpha1
|
||||
kind: Workflow
|
||||
metadata:
|
||||
generateName: uri-artifact-pipeline-
|
||||
annotations: {pipelines.kubeflow.org/kfp_sdk_version: 1.3.0, pipelines.kubeflow.org/pipeline_compilation_time: '2021-02-02T18:52:10.268063',
|
||||
pipelines.kubeflow.org/pipeline_spec: '{"inputs": [{"default": "Hello world!",
|
||||
"name": "text", "optional": true}], "name": "uri-artifact-pipeline"}'}
|
||||
labels: {pipelines.kubeflow.org/kfp_sdk_version: 1.3.0}
|
||||
spec:
|
||||
entrypoint: uri-artifact-pipeline
|
||||
templates:
|
||||
- name: condition-2
|
||||
inputs:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-}
|
||||
dag:
|
||||
tasks:
|
||||
- name: read-from-gcs-3
|
||||
template: read-from-gcs-3
|
||||
arguments:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-, value: '{{inputs.parameters.input_gcs_path-producer-pod-id-}}'}
|
||||
- name: flip-coin
|
||||
container:
|
||||
args: ['python -c "import random; result = ''heads'' if random.randint(0,1)
|
||||
== 0 else ''tails''; print(result)" | tee /tmp/output']
|
||||
command: [sh, -c]
|
||||
image: python:alpine3.6
|
||||
outputs:
|
||||
parameters:
|
||||
- name: flip-coin-output
|
||||
valueFrom: {path: /tmp/output}
|
||||
artifacts:
|
||||
- {name: flip-coin-output, path: /tmp/output}
|
||||
- name: for-loop-for-loop-1b89061c-1
|
||||
inputs:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-}
|
||||
dag:
|
||||
tasks:
|
||||
- name: read-from-gcs-2
|
||||
template: read-from-gcs-2
|
||||
arguments:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-, value: '{{inputs.parameters.input_gcs_path-producer-pod-id-}}'}
|
||||
- name: read-from-gcs
|
||||
container:
|
||||
args: []
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
gsutil cat "$0"
|
||||
- gs://gongyuan-pipeline-test/v2-in-v1/{{workflow.uid}}/{{inputs.parameters.input_gcs_path-producer-pod-id-}}/output_gcs_path
|
||||
image: google/cloud-sdk:slim
|
||||
inputs:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-}
|
||||
metadata:
|
||||
annotations: {pipelines.kubeflow.org/component_spec: '{"implementation": {"container":
|
||||
{"command": ["sh", "-c", "set -e -x\ngsutil cat \"$0\"\n", {"inputUri":
|
||||
"input_gcs_path"}], "image": "google/cloud-sdk:slim"}}, "inputs": [{"description":
|
||||
"GCS file path", "name": "input_gcs_path", "type": "String"}], "name": "Read
|
||||
from GCS"}', pipelines.kubeflow.org/component_ref: '{"digest": "d87d6c97f22a73ec8cd2e086a7727b2e46c391f09612da1b5001dfd9b824e5c7"}'}
|
||||
- name: read-from-gcs-2
|
||||
container:
|
||||
args: []
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
gsutil cat "$0"
|
||||
- gs://gongyuan-pipeline-test/v2-in-v1/{{workflow.uid}}/{{inputs.parameters.input_gcs_path-producer-pod-id-}}/output_gcs_path
|
||||
image: google/cloud-sdk:slim
|
||||
inputs:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-}
|
||||
metadata:
|
||||
annotations: {pipelines.kubeflow.org/component_spec: '{"implementation": {"container":
|
||||
{"command": ["sh", "-c", "set -e -x\ngsutil cat \"$0\"\n", {"inputUri":
|
||||
"input_gcs_path"}], "image": "google/cloud-sdk:slim"}}, "inputs": [{"description":
|
||||
"GCS file path", "name": "input_gcs_path", "type": "String"}], "name": "Read
|
||||
from GCS"}', pipelines.kubeflow.org/component_ref: '{"digest": "d87d6c97f22a73ec8cd2e086a7727b2e46c391f09612da1b5001dfd9b824e5c7"}'}
|
||||
- name: read-from-gcs-3
|
||||
container:
|
||||
args: []
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
gsutil cat "$0"
|
||||
- gs://gongyuan-pipeline-test/v2-in-v1/{{workflow.uid}}/{{inputs.parameters.input_gcs_path-producer-pod-id-}}/output_gcs_path
|
||||
image: google/cloud-sdk:slim
|
||||
inputs:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-}
|
||||
metadata:
|
||||
annotations: {pipelines.kubeflow.org/component_spec: '{"implementation": {"container":
|
||||
{"command": ["sh", "-c", "set -e -x\ngsutil cat \"$0\"\n", {"inputUri":
|
||||
"input_gcs_path"}], "image": "google/cloud-sdk:slim"}}, "inputs": [{"description":
|
||||
"GCS file path", "name": "input_gcs_path", "type": "String"}], "name": "Read
|
||||
from GCS"}', pipelines.kubeflow.org/component_ref: '{"digest": "d87d6c97f22a73ec8cd2e086a7727b2e46c391f09612da1b5001dfd9b824e5c7"}'}
|
||||
- name: uri-artifact-pipeline
|
||||
inputs:
|
||||
parameters:
|
||||
- {name: text}
|
||||
dag:
|
||||
tasks:
|
||||
- name: condition-2
|
||||
template: condition-2
|
||||
when: '"{{tasks.flip-coin.outputs.parameters.flip-coin-output}}" == "heads"'
|
||||
dependencies: [flip-coin, write-to-gcs]
|
||||
arguments:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-, value: '{{tasks.write-to-gcs.outputs.parameters.write-to-gcs-output_gcs_path-producer-pod-id-}}'}
|
||||
- {name: flip-coin, template: flip-coin}
|
||||
- name: for-loop-for-loop-1b89061c-1
|
||||
template: for-loop-for-loop-1b89061c-1
|
||||
dependencies: [write-to-gcs]
|
||||
arguments:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-, value: '{{tasks.write-to-gcs.outputs.parameters.write-to-gcs-output_gcs_path-producer-pod-id-}}'}
|
||||
withItems: [1, 2, 3, 4]
|
||||
- name: read-from-gcs
|
||||
template: read-from-gcs
|
||||
dependencies: [write-to-gcs]
|
||||
arguments:
|
||||
parameters:
|
||||
- {name: input_gcs_path-producer-pod-id-, value: '{{tasks.write-to-gcs.outputs.parameters.write-to-gcs-output_gcs_path-producer-pod-id-}}'}
|
||||
- name: write-to-gcs
|
||||
template: write-to-gcs
|
||||
arguments:
|
||||
parameters:
|
||||
- {name: text, value: '{{inputs.parameters.text}}'}
|
||||
- name: write-to-gcs
|
||||
container:
|
||||
args: []
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
echo "$0" | gsutil cp - "$1"
|
||||
- '{{inputs.parameters.text}}'
|
||||
- gs://gongyuan-pipeline-test/v2-in-v1/{{workflow.uid}}/{{pod.name}}/output_gcs_path
|
||||
image: google/cloud-sdk:slim
|
||||
inputs:
|
||||
parameters:
|
||||
- {name: text}
|
||||
outputs:
|
||||
parameters:
|
||||
- {name: write-to-gcs-output_gcs_path-producer-pod-id-, value: '{{pod.name}}'}
|
||||
metadata:
|
||||
annotations: {pipelines.kubeflow.org/component_spec: '{"implementation": {"container":
|
||||
{"command": ["sh", "-c", "set -e -x\necho \"$0\" | gsutil cp - \"$1\"\n",
|
||||
{"inputValue": "text"}, {"outputUri": "output_gcs_path"}], "image": "google/cloud-sdk:slim"}},
|
||||
"inputs": [{"description": "Content to be written to GCS", "name": "text",
|
||||
"type": "String"}], "name": "Write to GCS", "outputs": [{"description":
|
||||
"GCS file path", "name": "output_gcs_path", "type": "String"}]}', pipelines.kubeflow.org/component_ref: '{"digest":
|
||||
"aad0cd48834955bdd0d570a625283957f9c6ff3b77f59a2c47af933ae4b8ef64"}', pipelines.kubeflow.org/arguments.parameters: '{"text":
|
||||
"{{inputs.parameters.text}}"}'}
|
||||
arguments:
|
||||
parameters:
|
||||
- {name: text, value: Hello world!}
|
||||
serviceAccountName: pipeline-runner
|
||||
|
|
@ -0,0 +1,8 @@
|
|||
# temporary local data folder for simulating pipeline run
|
||||
tmp
|
||||
# compiled binary
|
||||
main
|
||||
# local env file to configure dev image repository
|
||||
.env
|
||||
# local token to identify whether image has been built up-to-date
|
||||
.build
|
||||
|
|
@ -0,0 +1,72 @@
|
|||
REPO_ROOT=../../../..
|
||||
GO_COMMON_ARGS=--logtostderr
|
||||
DRIVER_EXAMPLE_PATH=./driver/example
|
||||
PUBLISHER_EXAMPLE_PATH=./publisher/example
|
||||
DATA_PATH=./tmp
|
||||
DRIVER_CODE=./driver/*.go
|
||||
PUBLISHER_CODE=./publisher/*.go
|
||||
ENTRYPOINT_CODE=./entrypoint/*.go
|
||||
|
||||
.PHONY: run-producer-consumer
|
||||
run-producer-consumer: $(DRIVER_CODE) $(ENTRYPOINT_CODE) $(PUBLISHER_CODE) $(DRIVER_EXAMPLE_PATH)/task_spec_producer.json $(DRIVER_EXAMPLE_PATH)/task_spec_dag.json
|
||||
# Make sure you have a separate running terminal that runs
|
||||
# `make proxy`.
|
||||
|
||||
# Clean up intermediate data
|
||||
rm -rf $(DATA_PATH)
|
||||
@echo
|
||||
# ====== Run DAG driver ==========
|
||||
go run $(DRIVER_CODE) $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat $(DRIVER_EXAMPLE_PATH)/task_spec_dag.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=DAG \
|
||||
--output_path_execution_id="$(DATA_PATH)/dag/execution_id" \
|
||||
--output_path_context_name="$(DATA_PATH)/dag/context_name"
|
||||
@if ls $(DATA_PATH)/dag/execution_id $(DATA_PATH)/dag/context_name; then \
|
||||
echo "DAG driver outputs are found."; \
|
||||
else \
|
||||
echo "Some of DAG driver outputs are missing." \
|
||||
exit 1; \
|
||||
fi
|
||||
@echo
|
||||
# ======== Run producer task driver ========
|
||||
go run $(DRIVER_CODE) $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat $(DRIVER_EXAMPLE_PATH)/task_spec_producer.json)" \
|
||||
--executor_spec="$$(cat $(DRIVER_EXAMPLE_PATH)/executor_spec_producer.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=EXECUTOR \
|
||||
--parent_context_name="$$(cat $(DATA_PATH)/dag/context_name)" \
|
||||
--output_path_execution_id="$(DATA_PATH)/producer/execution_id" \
|
||||
--output_path_pod_spec_patch="$(DATA_PATH)/producer/pod_spec_patch.json"
|
||||
cat $(DATA_PATH)/producer/pod_spec_patch.json
|
||||
@echo
|
||||
@echo
|
||||
# ========= Run producer task entrypoint =======
|
||||
mkdir -p $(DATA_PATH)/producer/parameters
|
||||
go run $(ENTRYPOINT_CODE) $(GO_COMMON_ARGS) \
|
||||
--component_outputs_spec="$$(cat $(PUBLISHER_EXAMPLE_PATH)/outputs_spec.json)" \
|
||||
--execution_id=$$(cat $(DATA_PATH)/producer/execution_id) \
|
||||
--publisher_type=EXECUTOR \
|
||||
--input_path_parameters=$(DATA_PATH)/producer/parameters \
|
||||
-- "bash" "-c" "echo output value > $(DATA_PATH)/producer/parameters/output_value"
|
||||
@echo
|
||||
# ========= Run consumer task driver ==========
|
||||
go run $(DRIVER_CODE) $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat $(DRIVER_EXAMPLE_PATH)/task_spec_consumer.json)" \
|
||||
--executor_spec="$$(cat $(DRIVER_EXAMPLE_PATH)/executor_spec_consumer.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=EXECUTOR \
|
||||
--parent_context_name="$$(cat $(DATA_PATH)/dag/context_name)" \
|
||||
--output_path_execution_id="$(DATA_PATH)/consumer/execution_id" \
|
||||
--output_path_pod_spec_patch="$(DATA_PATH)/consumer/pod_spec_patch.json"
|
||||
cat $(DATA_PATH)/consumer/pod_spec_patch.json
|
||||
@echo
|
||||
|
||||
.PHONY: image-push
|
||||
image-push:
|
||||
cd driver && $(MAKE) image-push
|
||||
cd publisher && $(MAKE) image-push
|
||||
|
||||
.PHONY: proxy
|
||||
proxy:
|
||||
kubectl port-forward -n kubeflow svc/metadata-grpc-service 8080:8080
|
||||
|
|
@ -0,0 +1,16 @@
|
|||
# V2 Backend POC
|
||||
|
||||
* You can run this POC on a KFP cluster, take a look at `./compiler`.
|
||||
* You can run e2e testing for this POC locally by:
|
||||
|
||||
First, in one terminal, run the following to start a proxy to MLMD service in your KFP cluster (do not use your production cluster!):
|
||||
|
||||
```bash
|
||||
make proxy
|
||||
```
|
||||
|
||||
Then, run the e2e testing:
|
||||
|
||||
```bash
|
||||
make
|
||||
```
|
||||
|
|
@ -0,0 +1,33 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package common
|
||||
|
||||
// publisher type enum
|
||||
const (
|
||||
PublisherType_DAG = "DAG"
|
||||
PublisherType_EXECUTOR = "EXECUTOR"
|
||||
)
|
||||
|
||||
// executor output parameters path
|
||||
const (
|
||||
ExecutorOutputPathParameters = "/kfp/outputs/parameters"
|
||||
ExecutorEntrypointPath = "/kfp/entrypoint/entrypoint"
|
||||
ExecutorEntrypointVolumePath = "/kfp/entrypoint"
|
||||
)
|
||||
|
||||
// execution custom properties
|
||||
const (
|
||||
ExecutionPropertyPrefixOutputParam = "output:"
|
||||
)
|
||||
|
|
@ -0,0 +1,37 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package mlmd
|
||||
|
||||
import (
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/common"
|
||||
mlmdPb "github.com/kubeflow/pipelines/third_party/ml-metadata/go_client/ml_metadata/proto"
|
||||
)
|
||||
|
||||
// TODO(Bobgy): refactor driver and publisher to all use this helper
|
||||
type KfpExecution struct {
|
||||
execution *mlmdPb.Execution
|
||||
}
|
||||
|
||||
func NewKfpExecution(execution *mlmdPb.Execution) *KfpExecution {
|
||||
return &KfpExecution{execution: execution}
|
||||
}
|
||||
|
||||
func (e *KfpExecution) String() string {
|
||||
return e.execution.String()
|
||||
}
|
||||
|
||||
func (e *KfpExecution) GetOutputParameter(parameterName string) *mlmdPb.Value {
|
||||
return e.execution.GetCustomProperties()[common.ExecutionPropertyPrefixOutputParam+parameterName]
|
||||
}
|
||||
|
|
@ -0,0 +1,23 @@
|
|||
REPO_ROOT=../../../..
|
||||
# PIPELINE_SPEC_PATH=$(REPO_ROOT)/samples/v2/hello_world/hw_pipeline_job.json
|
||||
PIPELINE_SPEC_PATH=$(REPO_ROOT)/samples/v2/producer_consumer_param/two_step_pipeline_job.json
|
||||
SAMPLE_PIPELINE_SPEC=../driver/example/task_spec_dag.json
|
||||
|
||||
run: compile $(SAMPLE_PIPELINE_SPEC)
|
||||
# Run compiled argo workflow spec in workflow.yaml in an existing KFP cluster
|
||||
# using argo CLI to submit.
|
||||
#
|
||||
# Alternatively, you can upload this workflow.yaml in KFP UI and run it.
|
||||
argo submit --watch workflow.yaml \
|
||||
--serviceaccount pipeline-runner \
|
||||
--namespace kubeflow \
|
||||
--parameter task-spec="$$(cat $(SAMPLE_PIPELINE_SPEC))"
|
||||
|
||||
# TODO(Bobgy): How can we write ./**/*.go instead?
|
||||
.PHONY: compile
|
||||
compile: ./*.go ./templates/*.go
|
||||
set -o pipefail \
|
||||
&& ( \
|
||||
go run ./*.go --pipeline_spec_path=$(PIPELINE_SPEC_PATH) | tee workflow.yaml \
|
||||
|| (rm -f workflow.yaml && exit 1) \
|
||||
)
|
||||
|
|
@ -0,0 +1,134 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
workflowapi "github.com/argoproj/argo/pkg/apis/workflow/v1alpha1"
|
||||
pb "github.com/kubeflow/pipelines/api/v2alpha1/go"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/common"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/compiler/templates"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const (
|
||||
rootDagDriverTaskName = "driver-kfp-root"
|
||||
)
|
||||
|
||||
const (
|
||||
templateNameExecutorDriver = "kfp-executor-driver"
|
||||
templateNameDagDriver = "kfp-dag-driver"
|
||||
templateNameExecutorPublisher = "kfp-executor-publisher"
|
||||
)
|
||||
|
||||
func CompilePipelineSpec(
|
||||
pipelineSpec *pb.PipelineSpec,
|
||||
deploymentConfig *pb.PipelineDeploymentConfig,
|
||||
) (*workflowapi.Workflow, error) {
|
||||
|
||||
// validation
|
||||
if pipelineSpec.GetPipelineInfo().GetName() == "" {
|
||||
return nil, errors.New("Name is empty")
|
||||
}
|
||||
|
||||
// initialization
|
||||
var workflow workflowapi.Workflow
|
||||
workflow.APIVersion = "argoproj.io/v1alpha1"
|
||||
workflow.Kind = "Workflow"
|
||||
workflow.GenerateName = pipelineSpec.GetPipelineInfo().GetName() + "-"
|
||||
|
||||
spec, err := generateSpec(pipelineSpec, deploymentConfig)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to generate workflow spec")
|
||||
}
|
||||
workflow.Spec = *spec
|
||||
|
||||
return &workflow, nil
|
||||
}
|
||||
|
||||
func generateSpec(
|
||||
pipelineSpec *pb.PipelineSpec,
|
||||
deploymentConfig *pb.PipelineDeploymentConfig,
|
||||
) (*workflowapi.WorkflowSpec, error) {
|
||||
tasks := pipelineSpec.GetTasks()
|
||||
var spec workflowapi.WorkflowSpec
|
||||
|
||||
// generate helper templates
|
||||
executorDriver := templates.Driver(false)
|
||||
executorDriver.Name = templateNameExecutorDriver
|
||||
dagDriver := templates.Driver(true)
|
||||
dagDriver.Name = templateNameDagDriver
|
||||
executorPublisher := templates.Publisher(common.PublisherType_EXECUTOR)
|
||||
executorPublisher.Name = templateNameExecutorPublisher
|
||||
executorTemplates := templates.Executor(templateNameExecutorDriver, templateNameExecutorPublisher)
|
||||
|
||||
// generate root template
|
||||
var root workflowapi.Template
|
||||
root.Name = "kfp-root"
|
||||
rootDag := initRootDag(&spec, templateNameDagDriver)
|
||||
root.DAG = rootDag
|
||||
// TODO: make a generic default value
|
||||
defaultTaskSpec := `{"taskInfo":{"name":"hello-world-dag"},"inputs":{"parameters":{"text":{"runtimeValue":{"constantValue":{"stringValue":"Hello, World!"}}}}}}`
|
||||
|
||||
spec.Arguments.Parameters = []workflowapi.Parameter{
|
||||
{Name: "task-spec", Value: &defaultTaskSpec},
|
||||
}
|
||||
|
||||
subDag, err := templates.Dag(&templates.DagArgs{
|
||||
Tasks: &tasks,
|
||||
DeploymentConfig: deploymentConfig,
|
||||
ExecutorTemplateName: templates.TemplateNameExecutor,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
parentContextName := "{{tasks." + rootDagDriverTaskName + ".outputs.parameters." + templates.DriverParamContextName + "}}"
|
||||
root.DAG.Tasks = append(root.DAG.Tasks, workflowapi.DAGTask{
|
||||
Name: "sub-dag",
|
||||
Template: subDag.Name,
|
||||
Dependencies: []string{rootDagDriverTaskName},
|
||||
Arguments: workflowapi.Arguments{
|
||||
Parameters: []workflowapi.Parameter{
|
||||
{Name: templates.DagParamContextName, Value: &parentContextName},
|
||||
},
|
||||
},
|
||||
})
|
||||
|
||||
spec.Templates = []workflowapi.Template{root, *subDag, *executorDriver, *dagDriver, *executorPublisher}
|
||||
for _, template := range executorTemplates {
|
||||
spec.Templates = append(spec.Templates, *template)
|
||||
}
|
||||
spec.Entrypoint = root.Name
|
||||
return &spec, nil
|
||||
}
|
||||
|
||||
func initRootDag(spec *workflowapi.WorkflowSpec, templateNameDagDriver string) *workflowapi.DAGTemplate {
|
||||
root := &workflowapi.DAGTemplate{}
|
||||
// TODO(Bobgy): shall we pass a lambda "addTemplate()" here instead?
|
||||
driverTask := &workflowapi.DAGTask{}
|
||||
driverTask.Name = rootDagDriverTaskName
|
||||
driverTask.Template = templateNameDagDriver
|
||||
rootExecutionName := "kfp-root-{{workflow.name}}"
|
||||
workflowParameterTaskSpec := "{{workflow.parameters.task-spec}}"
|
||||
driverType := "DAG"
|
||||
parentContextName := "" // root has no parent
|
||||
driverTask.Arguments.Parameters = []workflowapi.Parameter{
|
||||
{Name: templates.DriverParamExecutionName, Value: &rootExecutionName},
|
||||
{Name: templates.DriverParamTaskSpec, Value: &workflowParameterTaskSpec},
|
||||
{Name: templates.DriverParamDriverType, Value: &driverType},
|
||||
{Name: templates.DriverParamParentContextName, Value: &parentContextName},
|
||||
}
|
||||
root.Tasks = append(root.Tasks, *driverTask)
|
||||
return root
|
||||
}
|
||||
|
|
@ -0,0 +1,89 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"flag"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
|
||||
"github.com/golang/glog"
|
||||
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
|
||||
"encoding/json"
|
||||
|
||||
pb "github.com/kubeflow/pipelines/api/v2alpha1/go"
|
||||
)
|
||||
|
||||
var pipelineSpecPath = flag.String("pipeline_spec_path", "../../../../samples/v2/hello_world/hw_pipeline_job.json", "Path to pipeline spec file generated by KFP SDK v2")
|
||||
|
||||
func main() {
|
||||
flag.Parse()
|
||||
if pipelineSpecPath == nil || *pipelineSpecPath == "" {
|
||||
glog.Fatal("Pipeline spec path is not provided")
|
||||
}
|
||||
in, err := ioutil.ReadFile(*pipelineSpecPath)
|
||||
if err != nil {
|
||||
fmt.Println(err)
|
||||
}
|
||||
jobJson := fmt.Sprintf("%s", in)
|
||||
// fmt.Println(specJson)
|
||||
job := &pb.PipelineJob{}
|
||||
if err := jsonpb.UnmarshalString(jobJson, job); err != nil {
|
||||
glog.Fatalf("Failed to parse pipeline job: %v, error: %s", jobJson, err)
|
||||
}
|
||||
// TODO(Bobgy): can we avoid this marshal to string step?
|
||||
marshaler := jsonpb.Marshaler{}
|
||||
specJson, err := marshaler.MarshalToString(job.GetPipelineSpec())
|
||||
if err != nil {
|
||||
glog.Fatalf("Failed marshal pipeline spec to json: %v", err)
|
||||
}
|
||||
spec := &pb.PipelineSpec{}
|
||||
if err := jsonpb.UnmarshalString(specJson, spec); err != nil {
|
||||
glog.Fatalf("Failed to parse pipeline spec: %v", err)
|
||||
}
|
||||
deploymentConfig, err := unmarshalDeploymentConfig(spec)
|
||||
if err != nil {
|
||||
glog.Fatalf("Failed to unmarshal deployment config: %v", err)
|
||||
}
|
||||
// fmt.Println(a)
|
||||
workflow, err := CompilePipelineSpec(spec, deploymentConfig)
|
||||
if err != nil {
|
||||
glog.Fatalf("Failed to compile pipeline IR to argo workflow: %v", err)
|
||||
}
|
||||
workflowInJson, err := json.MarshalIndent(workflow, "", " ")
|
||||
if err != nil {
|
||||
glog.Fatalf("Could not marshal the workflow: %v", err)
|
||||
}
|
||||
fmt.Println(string(workflowInJson))
|
||||
glog.Flush()
|
||||
}
|
||||
|
||||
func unmarshalDeploymentConfig(spec *pb.PipelineSpec) (*pb.PipelineDeploymentConfig, error) {
|
||||
marshaler := jsonpb.Marshaler{}
|
||||
buffer := new(bytes.Buffer)
|
||||
if err := marshaler.Marshal(buffer, spec.GetDeploymentConfig()); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
deploymentConfig := &pb.PipelineDeploymentConfig{}
|
||||
// Allow unknown '@type' field in the json message.
|
||||
unmarshaler := jsonpb.Unmarshaler{AllowUnknownFields: true}
|
||||
if err := unmarshaler.Unmarshal(buffer, deploymentConfig); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return deploymentConfig, nil
|
||||
}
|
||||
|
|
@ -0,0 +1,169 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package templates
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
workflowapi "github.com/argoproj/argo/pkg/apis/workflow/v1alpha1"
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
pb "github.com/kubeflow/pipelines/api/v2alpha1/go"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/compiler/util"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const (
|
||||
// Dag Inputs
|
||||
DagParamContextName = paramPrefixKfpInternal + "context-name"
|
||||
)
|
||||
|
||||
// Const can not be refered via string pointers, so we use var here.
|
||||
var (
|
||||
argoVariablePodName = "{{pod.name}}"
|
||||
)
|
||||
|
||||
type DagArgs struct {
|
||||
Tasks *[]*pb.PipelineTaskSpec
|
||||
DeploymentConfig *pb.PipelineDeploymentConfig
|
||||
ExecutorTemplateName string
|
||||
}
|
||||
|
||||
type taskData struct {
|
||||
task *pb.PipelineTaskSpec
|
||||
// we may need more stuff put here
|
||||
}
|
||||
|
||||
func Dag(args *DagArgs) (*workflowapi.Template, error) {
|
||||
// convenient local variables
|
||||
tasks := args.Tasks
|
||||
deploymentConfig := args.DeploymentConfig
|
||||
executors := deploymentConfig.GetExecutors()
|
||||
|
||||
var dag workflowapi.Template
|
||||
dag.Name = getUniqueDagName()
|
||||
dag.DAG = &workflowapi.DAGTemplate{}
|
||||
dag.Inputs.Parameters = []workflowapi.Parameter{
|
||||
{Name: DagParamContextName},
|
||||
}
|
||||
taskMap := make(map[string]*taskData)
|
||||
for index, task := range *tasks {
|
||||
name := task.GetTaskInfo().GetName()
|
||||
if name == "" {
|
||||
return nil, errors.Errorf("Task name is empty for task with index %v and spec: %s", index, task.String())
|
||||
}
|
||||
sanitizedName := util.SanitizeK8sName(name)
|
||||
if taskMap[sanitizedName] != nil {
|
||||
return nil, errors.Errorf("Two tasks '%s' and '%s' in the DAG has the same sanitized name: %s", taskMap[sanitizedName].task.GetTaskInfo().GetName(), name, sanitizedName)
|
||||
}
|
||||
taskMap[sanitizedName] = &taskData{
|
||||
task: task,
|
||||
}
|
||||
}
|
||||
|
||||
// generate tasks
|
||||
for _, task := range *tasks {
|
||||
// TODO(Bobgy): Move executor template generation out as a separate file.
|
||||
executorLabel := task.GetExecutorLabel()
|
||||
executorSpec := executors[executorLabel]
|
||||
if executorSpec == nil {
|
||||
return nil, errors.Errorf("Executor with label '%v' cannot be found in deployment config", executorLabel)
|
||||
}
|
||||
var executor workflowapi.Template
|
||||
executor.Name = util.SanitizeK8sName(executorLabel)
|
||||
|
||||
argoTaskName := util.SanitizeK8sName(task.GetTaskInfo().GetName())
|
||||
marshaler := &jsonpb.Marshaler{}
|
||||
taskSpecInJson, err := marshaler.MarshalToString(task)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to marshal task spec to JSON: %s", task.String())
|
||||
}
|
||||
executorSpecInJson, err := marshaler.MarshalToString(executorSpec)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to marshal executor spec to JSON: %s", executorSpec.String())
|
||||
}
|
||||
// TODO(Bobgy): Task outputs spec is deprecated. Get outputs spec from component output spec once data is ready.
|
||||
outputsSpec := task.GetOutputs()
|
||||
if outputsSpec == nil {
|
||||
// For tasks without outputs spec, marshal an emtpy outputs spec.
|
||||
outputsSpec = &pb.TaskOutputsSpec{}
|
||||
}
|
||||
outputsSpecInJson, err := marshaler.MarshalToString(outputsSpec)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to marshal outputs spec to JSON: %s", task.GetOutputs().String())
|
||||
}
|
||||
parentContextNameValue := "{{inputs.parameters." + DagParamContextName + "}}"
|
||||
|
||||
dependencies, err := getTaskDependencies(task.GetInputs())
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to get task dependencies for task: %s", task.String())
|
||||
}
|
||||
// Convert dependency names to sanitized ones and check validity.
|
||||
for index, dependency := range *dependencies {
|
||||
sanitizedDependencyName := util.SanitizeK8sName(dependency)
|
||||
upstreamTask := taskMap[sanitizedDependencyName]
|
||||
if upstreamTask == nil {
|
||||
return nil, errors.Wrapf(err, "Failed to find dependency '%s' for task: %s", dependency, task.String())
|
||||
}
|
||||
upstreamTaskName := upstreamTask.task.GetTaskInfo().GetName()
|
||||
if upstreamTaskName != dependency {
|
||||
return nil, errors.Wrapf(err, "Found slightly different dependency task name '%s', expecting '%s' for task: %s", upstreamTaskName, dependency, task.String())
|
||||
}
|
||||
(*dependencies)[index] = sanitizedDependencyName
|
||||
}
|
||||
|
||||
dag.DAG.Tasks = append(
|
||||
dag.DAG.Tasks,
|
||||
workflowapi.DAGTask{
|
||||
Name: argoTaskName,
|
||||
Template: args.ExecutorTemplateName,
|
||||
Dependencies: *dependencies,
|
||||
Arguments: workflowapi.Arguments{
|
||||
Parameters: []workflowapi.Parameter{
|
||||
{Name: ExecutorParamTaskSpec, Value: &taskSpecInJson},
|
||||
{Name: ExecutorParamContextName, Value: &parentContextNameValue},
|
||||
{Name: ExecutorParamExecutorSpec, Value: &executorSpecInJson},
|
||||
{Name: ExecutorParamOutputsSpec, Value: &outputsSpecInJson},
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
return &dag, nil
|
||||
}
|
||||
|
||||
func getTaskDependencies(inputsSpec *pb.TaskInputsSpec) (*[]string, error) {
|
||||
dependencies := make(map[string]bool)
|
||||
for _, parameter := range inputsSpec.GetParameters() {
|
||||
if parameter.GetTaskOutputParameter() != nil {
|
||||
producerTask := parameter.GetTaskOutputParameter().GetProducerTask()
|
||||
if producerTask == "" {
|
||||
return nil, errors.Errorf("Invalid task input parameter spec, producer task is empty: %v", parameter.String())
|
||||
}
|
||||
dependencies[producerTask] = true
|
||||
}
|
||||
}
|
||||
dependencyList := make([]string, 0, len(dependencies))
|
||||
for dependency := range dependencies {
|
||||
dependencyList = append(dependencyList, dependency)
|
||||
}
|
||||
return &dependencyList, nil
|
||||
}
|
||||
|
||||
// TODO(Bobgy): figure out a better way to generate unique names
|
||||
var globalDagCount = 0
|
||||
|
||||
func getUniqueDagName() string {
|
||||
globalDagCount = globalDagCount + 1
|
||||
return fmt.Sprintf("dag-%x", globalDagCount)
|
||||
}
|
||||
|
|
@ -0,0 +1,123 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package templates
|
||||
|
||||
import (
|
||||
workflowapi "github.com/argoproj/argo/pkg/apis/workflow/v1alpha1"
|
||||
k8sv1 "k8s.io/api/core/v1"
|
||||
)
|
||||
|
||||
// TODO(Bobgy): make image configurable
|
||||
// gcr.io/gongyuan-pipeline-test/kfp-driver:latest
|
||||
const (
|
||||
driverImage = "gcr.io/gongyuan-pipeline-test/kfp-driver"
|
||||
driverImageRef = "@sha256:d3fa780ffc59a22253eb4b4460e89f722811dfdd2ded277d3606f1fce323af87"
|
||||
driverImageFull = driverImage + driverImageRef
|
||||
)
|
||||
|
||||
const (
|
||||
paramPrefixKfpInternal = "kfp-"
|
||||
outputPathPodSpecPatch = "/kfp/outputs/pod-spec-patch.json"
|
||||
)
|
||||
|
||||
const (
|
||||
// Inputs
|
||||
DriverParamParentContextName = paramPrefixKfpInternal + "parent-context-name"
|
||||
DriverParamExecutionName = paramPrefixKfpInternal + "execution-name"
|
||||
DriverParamDriverType = paramPrefixKfpInternal + "driver-type"
|
||||
DriverParamTaskSpec = paramPrefixKfpInternal + "task-spec"
|
||||
DriverParamExecutorSpec = paramPrefixKfpInternal + "executor-spec"
|
||||
// Outputs
|
||||
DriverParamExecutionId = paramPrefixKfpInternal + "execution-id"
|
||||
DriverParamContextName = paramPrefixKfpInternal + "context-name"
|
||||
DriverParamPodSpecPatch = paramPrefixKfpInternal + "pod-spec-patch"
|
||||
)
|
||||
|
||||
// Do not modify this, this should be constant too.
|
||||
// This needs to be used in string pointers, so it is "var" instead of "const".
|
||||
var (
|
||||
mlmdExecutionName = "kfp-executor-{{pod.name}}"
|
||||
)
|
||||
|
||||
// TODO(Bobgy): parameters is no longer needed.
|
||||
// TODO(Bobgy): reuse existing templates if they are the same.
|
||||
func Driver(isDag bool) *workflowapi.Template {
|
||||
driver := &workflowapi.Template{}
|
||||
// driver.Name is not set, it should be set after calling this method.
|
||||
driver.Container = &k8sv1.Container{}
|
||||
driver.Container.Image = driverImageFull
|
||||
driver.Container.Command = []string{"/bin/kfp-driver"}
|
||||
driver.Inputs.Parameters = []workflowapi.Parameter{
|
||||
{Name: DriverParamParentContextName},
|
||||
{Name: DriverParamExecutionName, Value: &mlmdExecutionName},
|
||||
{Name: DriverParamDriverType},
|
||||
{Name: DriverParamTaskSpec},
|
||||
}
|
||||
driver.Outputs.Parameters = []workflowapi.Parameter{
|
||||
{Name: DriverParamExecutionId, ValueFrom: &workflowapi.ValueFrom{
|
||||
Path: "/kfp/outputs/internal/execution-id",
|
||||
}},
|
||||
}
|
||||
driver.Container.Args = []string{
|
||||
"--logtostderr",
|
||||
// TODO(Bobgy): make this configurable
|
||||
"--mlmd_url=metadata-grpc-service.kubeflow.svc.cluster.local:8080",
|
||||
"--parent_context_name={{inputs.parameters." + DriverParamParentContextName + "}}",
|
||||
"--execution_name={{inputs.parameters." + DriverParamExecutionName + "}}",
|
||||
"--driver_type={{inputs.parameters." + DriverParamDriverType + "}}",
|
||||
"--task_spec={{inputs.parameters." + DriverParamTaskSpec + "}}",
|
||||
"--output_path_execution_id={{outputs.parameters." + DriverParamExecutionId + ".path}}",
|
||||
}
|
||||
if isDag {
|
||||
driver.Container.Args = append(
|
||||
driver.Container.Args,
|
||||
"--output_path_context_name={{outputs.parameters."+DriverParamContextName+".path}}",
|
||||
)
|
||||
driver.Outputs.Parameters = append(
|
||||
driver.Outputs.Parameters,
|
||||
workflowapi.Parameter{
|
||||
Name: DriverParamContextName,
|
||||
ValueFrom: &workflowapi.ValueFrom{
|
||||
Path: "/kfp/outputs/internal/context-name",
|
||||
},
|
||||
},
|
||||
)
|
||||
} else {
|
||||
// input executor spec
|
||||
driver.Container.Args = append(
|
||||
driver.Container.Args,
|
||||
"--executor_spec={{inputs.parameters."+DriverParamExecutorSpec+"}}",
|
||||
)
|
||||
driver.Inputs.Parameters = append(
|
||||
driver.Inputs.Parameters,
|
||||
workflowapi.Parameter{Name: DriverParamExecutorSpec},
|
||||
)
|
||||
// output pod spec patch
|
||||
driver.Container.Args = append(
|
||||
driver.Container.Args,
|
||||
"--output_path_pod_spec_patch="+outputPathPodSpecPatch,
|
||||
)
|
||||
driver.Outputs.Parameters = append(
|
||||
driver.Outputs.Parameters,
|
||||
workflowapi.Parameter{
|
||||
Name: DriverParamPodSpecPatch,
|
||||
ValueFrom: &workflowapi.ValueFrom{
|
||||
Path: outputPathPodSpecPatch,
|
||||
},
|
||||
},
|
||||
)
|
||||
}
|
||||
return driver
|
||||
}
|
||||
|
|
@ -0,0 +1,134 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package templates
|
||||
|
||||
import (
|
||||
workflowapi "github.com/argoproj/argo/pkg/apis/workflow/v1alpha1"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/common"
|
||||
apiv1 "k8s.io/api/core/v1"
|
||||
)
|
||||
|
||||
// TODO(Bobgy): make image configurable
|
||||
const (
|
||||
entrypointImage = "gcr.io/gongyuan-pipeline-test/kfp-entrypoint"
|
||||
entrypointImageRef = "@sha256:793316de698fe53ab318d43b00213d947f5ac2abbefc780cece46bf8a44381c4"
|
||||
entrypointImageFull = entrypointImage + entrypointImageRef
|
||||
)
|
||||
|
||||
const (
|
||||
TemplateNameExecutor = "kfp-executor"
|
||||
templateNameDummy = "kfp-dummy"
|
||||
|
||||
// Executor inputs
|
||||
ExecutorParamTaskSpec = DriverParamTaskSpec
|
||||
ExecutorParamExecutorSpec = DriverParamExecutorSpec
|
||||
ExecutorParamContextName = paramPrefixKfpInternal + "context-name"
|
||||
ExecutorParamOutputsSpec = PublisherParamOutputsSpec
|
||||
|
||||
executorInternalParamPodName = paramPrefixKfpInternal + "pod-name"
|
||||
executorInternalParamPodSpecPatch = paramPrefixKfpInternal + "pod-spec-patch"
|
||||
executorInternalArtifactParameters = paramPrefixKfpInternal + "parameters"
|
||||
)
|
||||
|
||||
func Executor(executorDriverTemplateName string, executorPublisherTemplateName string) []*workflowapi.Template {
|
||||
// TODO(Bobgy): Move dummy template generation out as a separate file.
|
||||
var dummy workflowapi.Template
|
||||
dummy.Name = templateNameDummy
|
||||
// The actual container definition will be injected by pod spec patch
|
||||
entrypointVolumeName := "kfp-entrypoint"
|
||||
dummy.Volumes = []apiv1.Volume{{
|
||||
Name: entrypointVolumeName,
|
||||
VolumeSource: apiv1.VolumeSource{
|
||||
EmptyDir: &apiv1.EmptyDirVolumeSource{},
|
||||
},
|
||||
}}
|
||||
dummy.Container = &apiv1.Container{
|
||||
Image: "dummy",
|
||||
VolumeMounts: []apiv1.VolumeMount{{
|
||||
Name: entrypointVolumeName,
|
||||
MountPath: common.ExecutorEntrypointVolumePath,
|
||||
ReadOnly: true,
|
||||
}},
|
||||
}
|
||||
// The entrypoint init container copies the entrypoint binary into a volume shared with executor container.
|
||||
dummy.InitContainers = []workflowapi.UserContainer{{
|
||||
Container: apiv1.Container{
|
||||
Name: "entrypoint-init",
|
||||
Image: entrypointImageFull,
|
||||
Command: []string{"cp"},
|
||||
Args: []string{"/bin/kfp-entrypoint", common.ExecutorEntrypointPath},
|
||||
VolumeMounts: []apiv1.VolumeMount{{
|
||||
Name: entrypointVolumeName,
|
||||
MountPath: common.ExecutorEntrypointVolumePath,
|
||||
}},
|
||||
},
|
||||
}}
|
||||
dummy.PodSpecPatch = "{{inputs.parameters." + executorInternalParamPodSpecPatch + "}}"
|
||||
dummy.Inputs.Parameters = []workflowapi.Parameter{
|
||||
{Name: executorInternalParamPodSpecPatch},
|
||||
}
|
||||
dummy.Outputs.Parameters = []workflowapi.Parameter{
|
||||
{Name: executorInternalParamPodName, Value: &argoVariablePodName},
|
||||
}
|
||||
dummy.Outputs.Artifacts = []workflowapi.Artifact{
|
||||
{Name: executorInternalArtifactParameters, Path: common.ExecutorOutputPathParameters, Optional: true},
|
||||
}
|
||||
|
||||
driverTaskName := "driver"
|
||||
executorTaskName := "executor"
|
||||
driverType := "EXECUTOR"
|
||||
|
||||
var executorDag workflowapi.Template
|
||||
executorDag.Name = TemplateNameExecutor
|
||||
executorDag.Inputs.Parameters = []workflowapi.Parameter{
|
||||
{Name: ExecutorParamContextName},
|
||||
{Name: ExecutorParamTaskSpec},
|
||||
{Name: ExecutorParamExecutorSpec},
|
||||
{Name: ExecutorParamOutputsSpec},
|
||||
}
|
||||
taskSpecInJson := "{{inputs.parameters." + ExecutorParamTaskSpec + "}}"
|
||||
executorSpecInJson := "{{inputs.parameters." + ExecutorParamExecutorSpec + "}}"
|
||||
podSpecPatchValue := "{{tasks." + driverTaskName + ".outputs.parameters." + executorInternalParamPodSpecPatch + "}}"
|
||||
parentContextNameValue := "{{inputs.parameters." + ExecutorParamContextName + "}}"
|
||||
executorDag.DAG = &workflowapi.DAGTemplate{}
|
||||
executorDag.DAG.Tasks = []workflowapi.DAGTask{
|
||||
{
|
||||
Name: driverTaskName,
|
||||
Template: executorDriverTemplateName,
|
||||
Arguments: workflowapi.Arguments{
|
||||
Parameters: []workflowapi.Parameter{
|
||||
{Name: DriverParamTaskSpec, Value: &taskSpecInJson},
|
||||
{Name: DriverParamDriverType, Value: &driverType},
|
||||
{Name: DriverParamParentContextName, Value: &parentContextNameValue},
|
||||
{Name: DriverParamExecutorSpec, Value: &executorSpecInJson},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Name: executorTaskName,
|
||||
Template: dummy.Name,
|
||||
Dependencies: []string{driverTaskName},
|
||||
Arguments: workflowapi.Arguments{
|
||||
Parameters: []workflowapi.Parameter{
|
||||
{Name: executorInternalParamPodSpecPatch, Value: &podSpecPatchValue},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
return []*workflowapi.Template{
|
||||
&executorDag,
|
||||
&dummy,
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,68 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package templates
|
||||
|
||||
import (
|
||||
workflowapi "github.com/argoproj/argo/pkg/apis/workflow/v1alpha1"
|
||||
k8sv1 "k8s.io/api/core/v1"
|
||||
)
|
||||
|
||||
// TODO(Bobgy): make image configurable
|
||||
// gcr.io/gongyuan-pipeline-test/kfp-driver:latest
|
||||
const (
|
||||
publisherImage = "gcr.io/gongyuan-pipeline-test/kfp-publisher"
|
||||
publisherImageRef = "@sha256:3623052f6bd4f02da6199d1971ec948b96fb00c6d8c61725d3cfda998b304efe"
|
||||
publisherImageFull = publisherImage + publisherImageRef
|
||||
)
|
||||
|
||||
const (
|
||||
// Inputs
|
||||
PublisherParamExecutionId = paramPrefixKfpInternal + "execution-id"
|
||||
PublisherParamPublisherType = paramPrefixKfpInternal + "publisher-type"
|
||||
PublisherParamOutputsSpec = paramPrefixKfpInternal + "outputs-spec"
|
||||
PublisherArtifactParameters = paramPrefixKfpInternal + "parameters"
|
||||
// Paths
|
||||
publisherInputPathParameters = "/kfp/inputs/parameters"
|
||||
)
|
||||
|
||||
// TODO(Bobgy): parameters is no longer needed.
|
||||
// TODO(Bobgy): reuse existing templates if they are the same.
|
||||
func Publisher(publisherType string) *workflowapi.Template {
|
||||
publisher := &workflowapi.Template{}
|
||||
// publisher.Name is not set, it should be set after calling this method.
|
||||
publisher.Container = &k8sv1.Container{}
|
||||
publisher.Container.Image = publisherImageFull
|
||||
publisher.Container.Command = []string{"/bin/kfp-publisher"}
|
||||
publisher.Inputs.Parameters = []workflowapi.Parameter{
|
||||
{Name: PublisherParamExecutionId},
|
||||
{Name: PublisherParamOutputsSpec},
|
||||
{Name: PublisherParamPublisherType},
|
||||
}
|
||||
publisher.Inputs.Artifacts = []workflowapi.Artifact{{
|
||||
Name: PublisherArtifactParameters,
|
||||
Path: publisherInputPathParameters,
|
||||
Optional: true,
|
||||
}}
|
||||
publisher.Container.Args = []string{
|
||||
"--logtostderr",
|
||||
// TODO(Bobgy): make this configurable
|
||||
"--mlmd_url=metadata-grpc-service.kubeflow.svc.cluster.local:8080",
|
||||
"--execution_id={{inputs.parameters." + PublisherParamExecutionId + "}}",
|
||||
"--publisher_type={{inputs.parameters." + PublisherParamPublisherType + "}}",
|
||||
"--component_outputs_spec={{inputs.parameters." + PublisherParamOutputsSpec + "}}",
|
||||
"--input_path_parameters={{inputs.artifacts." + PublisherArtifactParameters + ".path}}",
|
||||
}
|
||||
return publisher
|
||||
}
|
||||
|
|
@ -0,0 +1,34 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package util
|
||||
|
||||
import (
|
||||
"regexp"
|
||||
"strings"
|
||||
)
|
||||
|
||||
func SanitizeK8sName(name string) string {
|
||||
expressionAlphaNumericOrHyphen := regexp.MustCompile(`[^-0-9a-z]+`)
|
||||
// replace non alphanumeric or hyphen characters to hyphen
|
||||
nameWithAlphaNumericAndHyphen :=
|
||||
expressionAlphaNumericOrHyphen.ReplaceAll(
|
||||
[]byte(strings.ToLower(name)),
|
||||
[]byte("-"))
|
||||
expressionMultipleHyphens := regexp.MustCompile(`-+`)
|
||||
// compress multiple hyphens to one
|
||||
return string(expressionMultipleHyphens.ReplaceAll(
|
||||
nameWithAlphaNumericAndHyphen,
|
||||
[]byte("-")))
|
||||
}
|
||||
|
|
@ -0,0 +1,446 @@
|
|||
# Copyright 2021 Google LLC
|
||||
#
|
||||
# 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.
|
||||
|
||||
{
|
||||
"kind": "Workflow",
|
||||
"apiVersion": "argoproj.io/v1alpha1",
|
||||
"metadata": {
|
||||
"generateName": "simple-two-step-pipeline-gongyuan-1609817103-",
|
||||
"creationTimestamp": null
|
||||
},
|
||||
"spec": {
|
||||
"templates": [
|
||||
{
|
||||
"name": "kfp-root",
|
||||
"arguments": {},
|
||||
"inputs": {},
|
||||
"outputs": {},
|
||||
"metadata": {},
|
||||
"dag": {
|
||||
"tasks": [
|
||||
{
|
||||
"name": "driver-kfp-root",
|
||||
"template": "kfp-dag-driver",
|
||||
"arguments": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-execution-name",
|
||||
"value": "kfp-root-{{workflow.name}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-task-spec",
|
||||
"value": "{{workflow.parameters.task-spec}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-driver-type",
|
||||
"value": "DAG"
|
||||
},
|
||||
{
|
||||
"name": "kfp-parent-context-name",
|
||||
"value": ""
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "sub-dag",
|
||||
"template": "dag-1",
|
||||
"arguments": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-context-name",
|
||||
"value": "{{tasks.driver-kfp-root.outputs.parameters.kfp-context-name}}"
|
||||
}
|
||||
]
|
||||
},
|
||||
"dependencies": [
|
||||
"driver-kfp-root"
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "dag-1",
|
||||
"arguments": {},
|
||||
"inputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-context-name"
|
||||
}
|
||||
]
|
||||
},
|
||||
"outputs": {},
|
||||
"metadata": {},
|
||||
"dag": {
|
||||
"tasks": [
|
||||
{
|
||||
"name": "producer",
|
||||
"template": "kfp-executor",
|
||||
"arguments": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-task-spec",
|
||||
"value": "{\"taskInfo\":{\"name\":\"Producer\"},\"inputs\":{\"parameters\":{\"text\":{\"componentInputParameter\":\"text\"}}},\"outputs\":{\"parameters\":{\"output_value\":{\"type\":\"STRING\"}}},\"executorLabel\":\"Producer\"}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-context-name",
|
||||
"value": "{{inputs.parameters.kfp-context-name}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor-spec",
|
||||
"value": "{\"container\":{\"image\":\"google/cloud-sdk:latest\",\"command\":[\"sh\",\"-c\",\"set -e -x\\necho \\\"$0, this is an output parameter\\\" | gsutil cp - \\\"$1\\\"\\n\",\"{{$.inputs.parameters['text']}}\",\"{{$.outputs.parameters['output_value'].output_file}}\"]}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-outputs-spec",
|
||||
"value": "{\"parameters\":{\"output_value\":{\"type\":\"STRING\"}}}"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "consumer",
|
||||
"template": "kfp-executor",
|
||||
"arguments": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-task-spec",
|
||||
"value": "{\"taskInfo\":{\"name\":\"Consumer\"},\"inputs\":{\"parameters\":{\"input_value\":{\"taskOutputParameter\":{\"producerTask\":\"Producer\",\"outputParameterKey\":\"output_value\"}}}},\"executorLabel\":\"Consumer\"}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-context-name",
|
||||
"value": "{{inputs.parameters.kfp-context-name}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor-spec",
|
||||
"value": "{\"container\":{\"image\":\"google/cloud-sdk:latest\",\"command\":[\"sh\",\"-c\",\"set -e -x\\necho \\\"Read from an input parameter: \\\" \\u0026\\u0026 echo \\\"$0\\\"\\n\",\"{{$.inputs.parameters['input_value']}}\"]}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-outputs-spec",
|
||||
"value": "{}"
|
||||
}
|
||||
]
|
||||
},
|
||||
"dependencies": [
|
||||
"producer"
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor-driver",
|
||||
"arguments": {},
|
||||
"inputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-parent-context-name"
|
||||
},
|
||||
{
|
||||
"name": "kfp-execution-name",
|
||||
"value": "kfp-executor-{{pod.name}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-driver-type"
|
||||
},
|
||||
{
|
||||
"name": "kfp-task-spec"
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor-spec"
|
||||
}
|
||||
]
|
||||
},
|
||||
"outputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-execution-id",
|
||||
"valueFrom": {
|
||||
"path": "/kfp/outputs/internal/execution-id"
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "kfp-pod-spec-patch",
|
||||
"valueFrom": {
|
||||
"path": "/kfp/outputs/pod-spec-patch.json"
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"metadata": {},
|
||||
"container": {
|
||||
"name": "",
|
||||
"image": "gcr.io/gongyuan-pipeline-test/kfp-driver@sha256:d3fa780ffc59a22253eb4b4460e89f722811dfdd2ded277d3606f1fce323af87",
|
||||
"command": [
|
||||
"/bin/kfp-driver"
|
||||
],
|
||||
"args": [
|
||||
"--logtostderr",
|
||||
"--mlmd_url=metadata-grpc-service.kubeflow.svc.cluster.local:8080",
|
||||
"--parent_context_name={{inputs.parameters.kfp-parent-context-name}}",
|
||||
"--execution_name={{inputs.parameters.kfp-execution-name}}",
|
||||
"--driver_type={{inputs.parameters.kfp-driver-type}}",
|
||||
"--task_spec={{inputs.parameters.kfp-task-spec}}",
|
||||
"--output_path_execution_id={{outputs.parameters.kfp-execution-id.path}}",
|
||||
"--executor_spec={{inputs.parameters.kfp-executor-spec}}",
|
||||
"--output_path_pod_spec_patch=/kfp/outputs/pod-spec-patch.json"
|
||||
],
|
||||
"resources": {}
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "kfp-dag-driver",
|
||||
"arguments": {},
|
||||
"inputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-parent-context-name"
|
||||
},
|
||||
{
|
||||
"name": "kfp-execution-name",
|
||||
"value": "kfp-executor-{{pod.name}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-driver-type"
|
||||
},
|
||||
{
|
||||
"name": "kfp-task-spec"
|
||||
}
|
||||
]
|
||||
},
|
||||
"outputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-execution-id",
|
||||
"valueFrom": {
|
||||
"path": "/kfp/outputs/internal/execution-id"
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "kfp-context-name",
|
||||
"valueFrom": {
|
||||
"path": "/kfp/outputs/internal/context-name"
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"metadata": {},
|
||||
"container": {
|
||||
"name": "",
|
||||
"image": "gcr.io/gongyuan-pipeline-test/kfp-driver@sha256:d3fa780ffc59a22253eb4b4460e89f722811dfdd2ded277d3606f1fce323af87",
|
||||
"command": [
|
||||
"/bin/kfp-driver"
|
||||
],
|
||||
"args": [
|
||||
"--logtostderr",
|
||||
"--mlmd_url=metadata-grpc-service.kubeflow.svc.cluster.local:8080",
|
||||
"--parent_context_name={{inputs.parameters.kfp-parent-context-name}}",
|
||||
"--execution_name={{inputs.parameters.kfp-execution-name}}",
|
||||
"--driver_type={{inputs.parameters.kfp-driver-type}}",
|
||||
"--task_spec={{inputs.parameters.kfp-task-spec}}",
|
||||
"--output_path_execution_id={{outputs.parameters.kfp-execution-id.path}}",
|
||||
"--output_path_context_name={{outputs.parameters.kfp-context-name.path}}"
|
||||
],
|
||||
"resources": {}
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor-publisher",
|
||||
"arguments": {},
|
||||
"inputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-execution-id"
|
||||
},
|
||||
{
|
||||
"name": "kfp-outputs-spec"
|
||||
},
|
||||
{
|
||||
"name": "kfp-publisher-type"
|
||||
}
|
||||
],
|
||||
"artifacts": [
|
||||
{
|
||||
"name": "kfp-parameters",
|
||||
"path": "/kfp/inputs/parameters",
|
||||
"optional": true
|
||||
}
|
||||
]
|
||||
},
|
||||
"outputs": {},
|
||||
"metadata": {},
|
||||
"container": {
|
||||
"name": "",
|
||||
"image": "gcr.io/gongyuan-pipeline-test/kfp-publisher@sha256:3623052f6bd4f02da6199d1971ec948b96fb00c6d8c61725d3cfda998b304efe",
|
||||
"command": [
|
||||
"/bin/kfp-publisher"
|
||||
],
|
||||
"args": [
|
||||
"--logtostderr",
|
||||
"--mlmd_url=metadata-grpc-service.kubeflow.svc.cluster.local:8080",
|
||||
"--execution_id={{inputs.parameters.kfp-execution-id}}",
|
||||
"--publisher_type={{inputs.parameters.kfp-publisher-type}}",
|
||||
"--component_outputs_spec={{inputs.parameters.kfp-outputs-spec}}",
|
||||
"--input_path_parameters={{inputs.artifacts.kfp-parameters.path}}"
|
||||
],
|
||||
"resources": {}
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor",
|
||||
"arguments": {},
|
||||
"inputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-context-name"
|
||||
},
|
||||
{
|
||||
"name": "kfp-task-spec"
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor-spec"
|
||||
},
|
||||
{
|
||||
"name": "kfp-outputs-spec"
|
||||
}
|
||||
]
|
||||
},
|
||||
"outputs": {},
|
||||
"metadata": {},
|
||||
"dag": {
|
||||
"tasks": [
|
||||
{
|
||||
"name": "driver",
|
||||
"template": "kfp-executor-driver",
|
||||
"arguments": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-task-spec",
|
||||
"value": "{{inputs.parameters.kfp-task-spec}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-driver-type",
|
||||
"value": "EXECUTOR"
|
||||
},
|
||||
{
|
||||
"name": "kfp-parent-context-name",
|
||||
"value": "{{inputs.parameters.kfp-context-name}}"
|
||||
},
|
||||
{
|
||||
"name": "kfp-executor-spec",
|
||||
"value": "{{inputs.parameters.kfp-executor-spec}}"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "executor",
|
||||
"template": "kfp-dummy",
|
||||
"arguments": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-pod-spec-patch",
|
||||
"value": "{{tasks.driver.outputs.parameters.kfp-pod-spec-patch}}"
|
||||
}
|
||||
]
|
||||
},
|
||||
"dependencies": [
|
||||
"driver"
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "kfp-dummy",
|
||||
"arguments": {},
|
||||
"inputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-pod-spec-patch"
|
||||
}
|
||||
]
|
||||
},
|
||||
"outputs": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "kfp-pod-name",
|
||||
"value": "{{pod.name}}"
|
||||
}
|
||||
],
|
||||
"artifacts": [
|
||||
{
|
||||
"name": "kfp-parameters",
|
||||
"path": "/kfp/outputs/parameters",
|
||||
"optional": true
|
||||
}
|
||||
]
|
||||
},
|
||||
"metadata": {},
|
||||
"container": {
|
||||
"name": "",
|
||||
"image": "dummy",
|
||||
"resources": {},
|
||||
"volumeMounts": [
|
||||
{
|
||||
"name": "kfp-entrypoint",
|
||||
"readOnly": true,
|
||||
"mountPath": "/kfp/entrypoint"
|
||||
}
|
||||
]
|
||||
},
|
||||
"volumes": [
|
||||
{
|
||||
"name": "kfp-entrypoint",
|
||||
"emptyDir": {}
|
||||
}
|
||||
],
|
||||
"initContainers": [
|
||||
{
|
||||
"name": "entrypoint-init",
|
||||
"image": "gcr.io/gongyuan-pipeline-test/kfp-entrypoint@sha256:793316de698fe53ab318d43b00213d947f5ac2abbefc780cece46bf8a44381c4",
|
||||
"command": [
|
||||
"cp"
|
||||
],
|
||||
"args": [
|
||||
"/bin/kfp-entrypoint",
|
||||
"/kfp/entrypoint/entrypoint"
|
||||
],
|
||||
"resources": {},
|
||||
"volumeMounts": [
|
||||
{
|
||||
"name": "kfp-entrypoint",
|
||||
"mountPath": "/kfp/entrypoint"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"podSpecPatch": "{{inputs.parameters.kfp-pod-spec-patch}}"
|
||||
}
|
||||
],
|
||||
"entrypoint": "kfp-root",
|
||||
"arguments": {
|
||||
"parameters": [
|
||||
{
|
||||
"name": "task-spec",
|
||||
"value": "{\"taskInfo\":{\"name\":\"hello-world-dag\"},\"inputs\":{\"parameters\":{\"text\":{\"runtimeValue\":{\"constantValue\":{\"stringValue\":\"Hello, World!\"}}}}}}"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"status": {
|
||||
"startedAt": null,
|
||||
"finishedAt": null
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,2 @@
|
|||
inputs_spec.json
|
||||
data
|
||||
|
|
@ -0,0 +1,4 @@
|
|||
FROM scratch
|
||||
WORKDIR /bin
|
||||
COPY main /bin/kfp-driver
|
||||
ENTRYPOINT ["/bin/kfp-driver"]
|
||||
|
|
@ -0,0 +1,106 @@
|
|||
REPO_ROOT=../../../..
|
||||
GO_COMMON_ARGS=--logtostderr
|
||||
|
||||
# Example .env file:
|
||||
# ```
|
||||
# IMAGE_DEV=gcr.io/gongyuan-pipeline-test/kfp-publisher
|
||||
# ```
|
||||
# It should be an image repository where you can push to via docker push.
|
||||
include .env
|
||||
|
||||
.PHONY: run-producer
|
||||
run-producer: *.go ./example/task_spec_dag.json ./example/task_spec_producer.json ./example/executor_spec_producer.json
|
||||
# Make sure you have a separate running terminal that runs
|
||||
# `make proxy`.
|
||||
|
||||
# Clean up intermediate data
|
||||
rm -rf ./data
|
||||
# Run DAG driver
|
||||
go run *.go $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat ./example/task_spec_dag.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=DAG \
|
||||
--output_path_execution_id="./data/execution_id" \
|
||||
--output_path_context_name="./data/context_name"
|
||||
# Run task driver
|
||||
@if ls ./example/task_spec_producer.json ./data/execution_id ./data/context_name; then \
|
||||
echo "inputs are found"; \
|
||||
else \
|
||||
exit 1; \
|
||||
fi
|
||||
go run *.go $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat ./example/task_spec_producer.json)" \
|
||||
--executor_spec="$$(cat ./example/executor_spec_producer.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=EXECUTOR \
|
||||
--parent_context_name="$$(cat ./data/context_name)" \
|
||||
--output_path_execution_id="./data/execution_id" \
|
||||
--output_path_pod_spec_patch="./data/pod_spec_patch.json"
|
||||
cat ./data/pod_spec_patch.json
|
||||
|
||||
.PHONY: run
|
||||
run: *.go example/task_spec_dag.json
|
||||
# Make sure you have a separate running terminal that runs
|
||||
# `make proxy`.
|
||||
|
||||
# Clean up intermediate data
|
||||
rm -rf ./data
|
||||
# Run DAG driver
|
||||
go run *.go $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat ./example/task_spec_dag.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=DAG \
|
||||
--output_path_execution_id="./data/execution_id" \
|
||||
--output_path_context_name="./data/context_name"
|
||||
# Run task driver
|
||||
@if ls ./example/task_spec_hw.json ./data/execution_id ./data/context_name; then \
|
||||
echo "inputs are found"; \
|
||||
else \
|
||||
exit 1; \
|
||||
fi
|
||||
go run *.go $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat ./example/task_spec_hw.json)" \
|
||||
--executor_spec="$$(cat ./example/executor_spec_hw.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=EXECUTOR \
|
||||
--parent_context_name="$$(cat ./data/context_name)" \
|
||||
--output_path_execution_id="./data/execution_id" \
|
||||
--output_path_pod_spec_patch="./data/pod_spec_patch.json"
|
||||
cat ./data/pod_spec_patch.json
|
||||
|
||||
.PHONY: run-docker
|
||||
run-docker: .build
|
||||
docker run kfp/driver $(GO_COMMON_ARGS) \
|
||||
--task_spec="$$(cat ./example/task_spec_dag.json)" \
|
||||
--execution_name="kfp-$$RANDOM" \
|
||||
--driver_type=DAG \
|
||||
--output_path_execution_id="./data/execution_id" \
|
||||
--output_path_context_name="./data/context_name" \
|
||||
--mlmd_url="host.docker.internal:8080"
|
||||
|
||||
.PHONY: proxy
|
||||
proxy:
|
||||
kubectl port-forward -n kubeflow svc/metadata-grpc-service 8080:8080
|
||||
|
||||
.PHONY: example
|
||||
example: example/task_spec_hw.json
|
||||
|
||||
.PHONY: image
|
||||
image: .build
|
||||
|
||||
.PHONY: image-push
|
||||
image-push: image
|
||||
docker push $(IMAGE_DEV)
|
||||
|
||||
.build: Dockerfile.dev main .env
|
||||
docker build . -f Dockerfile.dev -t kfp/driver -t $(IMAGE_DEV)
|
||||
touch .build
|
||||
|
||||
main: *.go $(REPO_ROOT)/go.mod
|
||||
env GOOS=linux CGO_ENABLED=0 go build -ldflags '-extldflags "-static"' -o main *.go
|
||||
|
||||
example/task_spec_dag.json: example/*.go
|
||||
go run example/*.go
|
||||
|
||||
example/task_spec_hw.json: example/*.go
|
||||
go run example/*.go
|
||||
|
|
@ -0,0 +1,25 @@
|
|||
# KFP Driver
|
||||
|
||||
## Responsibility
|
||||
|
||||
* Read KFP task's context from MLMD.
|
||||
* Init MLMD execution.
|
||||
* Resolve input artifacts and parameters and output them through argo parameters.
|
||||
|
||||
## Pseudo Code
|
||||
|
||||
```go
|
||||
func driver(parentExecutionId int, taskSpec Spec) {
|
||||
mlmdClient := newMlmdClient()
|
||||
parentExecution := mlmdClient.GetExecutionById(parentExecutionId)
|
||||
parentParameters := parentExecution.GetParameters()
|
||||
for _, paramName, paramSource := taskSpec.Inputs.Parameters {
|
||||
if paramSource == runtimeValueOrParameter {
|
||||
ioutil.write(
|
||||
parentExecution.GetCustomProperty("input:"+paramName),
|
||||
"/kfp/parameters/" + paramName
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
|
@ -0,0 +1,120 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"io/ioutil"
|
||||
|
||||
"github.com/golang/glog"
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
pb "github.com/kubeflow/pipelines/api/v2alpha1/go"
|
||||
)
|
||||
|
||||
const (
|
||||
outputPath = "./example"
|
||||
)
|
||||
|
||||
func exampleTaskSpec_DAG() {
|
||||
taskSpec := &pb.PipelineTaskSpec{}
|
||||
taskSpec.TaskInfo = &pb.PipelineTaskInfo{
|
||||
Name: "hello-world-dag",
|
||||
}
|
||||
taskInputsSpec := &pb.TaskInputsSpec{}
|
||||
taskSpec.Inputs = taskInputsSpec
|
||||
value := &pb.TaskInputsSpec_InputParameterSpec{}
|
||||
// This wrapper hell might be improved when issues like https://github.com/golang/protobuf/issues/283
|
||||
// is resolved.
|
||||
// TODO(Bobgy): investigate if there are better workarounds we can use now.
|
||||
value.Kind = &pb.TaskInputsSpec_InputParameterSpec_RuntimeValue{
|
||||
RuntimeValue: &pb.ValueOrRuntimeParameter{
|
||||
Value: &pb.ValueOrRuntimeParameter_ConstantValue{
|
||||
ConstantValue: &pb.Value{Value: &pb.Value_StringValue{StringValue: "Hello, World!"}},
|
||||
}}}
|
||||
taskInputsSpec.Parameters = make(map[string]*pb.TaskInputsSpec_InputParameterSpec)
|
||||
taskInputsSpec.Parameters["text"] = value
|
||||
|
||||
marshaler := &jsonpb.Marshaler{}
|
||||
jsonSpec, _ := marshaler.MarshalToString(taskSpec)
|
||||
// jsonSpec, _ := json.MarshalIndent(taskInputsSpec, "", " ")
|
||||
err := ioutil.WriteFile(outputPath+"/task_spec_dag.json", []byte(jsonSpec), 0644)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
// fmt.Printf("%s\n", jsonSpec)
|
||||
}
|
||||
|
||||
func exampleTaskSpec_HelloWorld() {
|
||||
taskSpec := &pb.PipelineTaskSpec{}
|
||||
taskSpec.TaskInfo = &pb.PipelineTaskInfo{
|
||||
Name: "hello-world-task",
|
||||
}
|
||||
taskInputsSpec := &pb.TaskInputsSpec{}
|
||||
taskSpec.Inputs = taskInputsSpec
|
||||
value := &pb.TaskInputsSpec_InputParameterSpec{}
|
||||
// This wrapper hell might be improved when issues like https://github.com/golang/protobuf/issues/283
|
||||
// is resolved.
|
||||
// TODO(Bobgy): investigate if there are better workarounds we can use now.
|
||||
value.Kind = &pb.TaskInputsSpec_InputParameterSpec_ComponentInputParameter{
|
||||
ComponentInputParameter: "text",
|
||||
}
|
||||
taskInputsSpec.Parameters = make(map[string]*pb.TaskInputsSpec_InputParameterSpec)
|
||||
taskInputsSpec.Parameters["text"] = value
|
||||
|
||||
marshaler := &jsonpb.Marshaler{}
|
||||
jsonSpec, _ := marshaler.MarshalToString(taskSpec)
|
||||
// jsonSpec, _ := json.MarshalIndent(taskInputsSpec, "", " ")
|
||||
err := ioutil.WriteFile(outputPath+"/task_spec_hw.json", []byte(jsonSpec), 0644)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
// fmt.Printf("%s\n", jsonSpec)
|
||||
}
|
||||
|
||||
func exampleExecutorSpec_HelloWorld() {
|
||||
executorSpecJson := `
|
||||
{
|
||||
"container": {
|
||||
"image": "python:3.7",
|
||||
"args": ["--text", "{{$.inputs.parameters['text']}}"],
|
||||
"command": [
|
||||
"python3",
|
||||
"-u",
|
||||
"-c",
|
||||
"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"
|
||||
]
|
||||
}
|
||||
}`
|
||||
var executorSpec pb.PipelineDeploymentConfig_ExecutorSpec
|
||||
// verify the json string has correct format
|
||||
err := jsonpb.UnmarshalString(executorSpecJson, &executorSpec)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
marshaler := &jsonpb.Marshaler{}
|
||||
jsonSpec, err := marshaler.MarshalToString(&executorSpec)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
err = ioutil.WriteFile(outputPath+"/executor_spec_hw.json", []byte(jsonSpec), 0644)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func main() {
|
||||
exampleTaskSpec_DAG()
|
||||
exampleTaskSpec_HelloWorld()
|
||||
exampleExecutorSpec_HelloWorld()
|
||||
}
|
||||
|
|
@ -0,0 +1,11 @@
|
|||
{
|
||||
"container": {
|
||||
"image": "google/cloud-sdk:latest",
|
||||
"command": [
|
||||
"sh",
|
||||
"-c",
|
||||
"set -e -x\necho \"Read from an input parameter: \" && echo \"$0\"\n",
|
||||
"{{$.inputs.parameters['input_value']}}"
|
||||
]
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1 @@
|
|||
{"container":{"image":"python:3.7","command":["python3","-u","-c","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"],"args":["--text","{{$.inputs.parameters['text']}}"]}}
|
||||
|
|
@ -0,0 +1,13 @@
|
|||
{
|
||||
"container": {
|
||||
"command": [
|
||||
"sh",
|
||||
"-c",
|
||||
"set -e -x\necho \"$0, this is an output parameter\" | gsutil cp - \"$1\"\necho \"$0, this is an output artifact\" | gsutil cp - \"$2\"\n",
|
||||
"{{$.inputs.parameters['input_text']}}",
|
||||
"{{$.outputs.parameters['output_value'].output_file}}",
|
||||
"{{$.outputs.artifacts['output_artifact'].uri}}"
|
||||
],
|
||||
"image": "google/cloud-sdk:latest"
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1 @@
|
|||
{"parameters":{"text":{"runtimeValue":{"constantValue":{"stringValue":"Hello"}}}}}
|
||||
|
|
@ -0,0 +1 @@
|
|||
{"parameters":{"text":{"componentInputParameter":"text"}}}
|
||||
|
|
@ -0,0 +1,16 @@
|
|||
{
|
||||
"executorLabel": "Consumer",
|
||||
"inputs": {
|
||||
"parameters": {
|
||||
"input_value": {
|
||||
"taskOutputParameter": {
|
||||
"producerTask": "Producer",
|
||||
"outputParameterKey": "output_value"
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"taskInfo": {
|
||||
"name": "Consumer"
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1 @@
|
|||
{"taskInfo":{"name":"hello-world-dag"},"inputs":{"parameters":{"text":{"runtimeValue":{"constantValue":{"stringValue":"Hello, World!"}}}}}}
|
||||
|
|
@ -0,0 +1,4 @@
|
|||
{
|
||||
"taskInfo": { "name": "hello-world-task" },
|
||||
"inputs": { "parameters": { "text": { "componentInputParameter": "text" } } }
|
||||
}
|
||||
|
|
@ -0,0 +1,25 @@
|
|||
{
|
||||
"outputs": {
|
||||
"parameters": {
|
||||
"output_value": {
|
||||
"type": "STRING"
|
||||
}
|
||||
},
|
||||
"artifacts": {
|
||||
"output_artifact": {
|
||||
"artifactType": {
|
||||
"instanceSchema": "properties:\ntitle: kfp.Artifact\ntype: object\n"
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"executorLabel": "Producer",
|
||||
"inputs": {
|
||||
"parameters": {
|
||||
"text": { "componentInputParameter": "text" }
|
||||
}
|
||||
},
|
||||
"taskInfo": {
|
||||
"name": "Producer"
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,117 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/golang/glog"
|
||||
)
|
||||
|
||||
// command line arguments
|
||||
const (
|
||||
argumentMlmdUrl = "mlmd_url"
|
||||
argumentMlmdUrlDefault = "localhost:8080"
|
||||
argumentTaskSpec = "task_spec"
|
||||
argumentExecutorSpec = "executor_spec"
|
||||
argumentExecutionName = "execution_name"
|
||||
argumentDriverType = "driver_type"
|
||||
argumentOutputPathExecutionId = "output_path_execution_id"
|
||||
argumentOutputPathContextName = "output_path_context_name"
|
||||
argumentOutputPathParameters = "output_path_parameters"
|
||||
argumentOutputPathPodSpecPatch = "output_path_pod_spec_patch"
|
||||
argumentParentContextName = "parent_context_name"
|
||||
)
|
||||
|
||||
// command line variables
|
||||
var (
|
||||
mlmdUrl string
|
||||
taskSpecJson string
|
||||
executorSpecJson string
|
||||
executionName string
|
||||
driverType string
|
||||
parentContextName string
|
||||
outputPathExecutionId string
|
||||
outputPathContextName string
|
||||
outputPathParameters string
|
||||
outputPathPodSpecPatch string
|
||||
)
|
||||
|
||||
// driver type enum
|
||||
const (
|
||||
driverTypeDag = "DAG"
|
||||
driverTypeExecutor = "EXECUTOR"
|
||||
)
|
||||
|
||||
func initFlags() {
|
||||
flag.StringVar(&mlmdUrl, argumentMlmdUrl, argumentMlmdUrlDefault, "URL of MLMD, defaults to localhost:8080")
|
||||
flag.StringVar(&taskSpecJson, argumentTaskSpec, "", "PipelineTaskSpec")
|
||||
// TODO(Bobgy): add component spec
|
||||
flag.StringVar(&executionName, argumentExecutionName, "", "Unique execution name")
|
||||
flag.StringVar(&driverType, argumentDriverType, "", fmt.Sprintf("Driver type, can be '%s' or '%s'", driverTypeDag, driverTypeExecutor))
|
||||
flag.StringVar(&parentContextName, argumentParentContextName, "", "Name of parent context. Required if not root DAG.")
|
||||
flag.StringVar(&outputPathExecutionId, argumentOutputPathExecutionId, "", "Output path where execution ID should be written to")
|
||||
|
||||
// Required when driving a DAG.
|
||||
flag.StringVar(&outputPathContextName, argumentOutputPathContextName, "", "Output path where context name should be written to. Required when driver type is DAG.")
|
||||
|
||||
// Required when driving an executor.
|
||||
flag.StringVar(&executorSpecJson, argumentExecutorSpec, "", "ExecutorSpec. Required when driver type is EXECUTOR.")
|
||||
// TODO(Bobgy): this will not be used most likely, keep it here for now.
|
||||
flag.StringVar(&outputPathParameters, argumentOutputPathParameters, "", "Output path where parameters should be written to.")
|
||||
flag.StringVar(&outputPathPodSpecPatch, argumentOutputPathPodSpecPatch, "", "Output path where pod spec patch should be written to. Required when driver type is EXECUTOR.")
|
||||
|
||||
flag.Parse()
|
||||
glog.Infof("Driver arguments: %v", os.Args)
|
||||
}
|
||||
|
||||
func validateFlagsOrFatal() {
|
||||
if driverType == "" {
|
||||
glog.Fatalln(argumentDriverType + " is empty.")
|
||||
} else if driverType != driverTypeDag && driverType != driverTypeExecutor {
|
||||
glog.Fatalf("invalid %s provided: %s. It should be either '%s' or '%s'", argumentDriverType, driverType, driverTypeDag, driverTypeExecutor)
|
||||
}
|
||||
if executionName == "" {
|
||||
glog.Fatalln(argumentExecutionName + " is empty.")
|
||||
}
|
||||
if taskSpecJson == "" {
|
||||
glog.Fatalln(argumentTaskSpec + " is empty.")
|
||||
}
|
||||
if driverType == driverTypeExecutor && executorSpecJson == "" {
|
||||
glog.Fatalln(argumentExecutorSpec + " is empty. It is required for task drivers.")
|
||||
}
|
||||
if outputPathExecutionId == "" {
|
||||
glog.Fatalln(argumentOutputPathExecutionId + " is empty.")
|
||||
}
|
||||
// Context name is only produced when driving a DAG.
|
||||
if driverType == driverTypeDag && outputPathContextName == "" {
|
||||
glog.Fatalf("%s is empty. Required when driver type is %s.", argumentOutputPathContextName, driverTypeDag)
|
||||
}
|
||||
if driverType == driverTypeExecutor {
|
||||
// Temporarily commented out, because there's no decision yet, whether
|
||||
// drivers need to output parameters or not.
|
||||
|
||||
// Parameters are only produced when driving type is EXECUTOR.
|
||||
// if outputPathParameters == "" {
|
||||
// glog.Fatalf("%s is empty. Required when driver type is %s.", argumentOutputPathParameters, driverTypeExecutor)
|
||||
// }
|
||||
// Pod spec patch is only produced when driver type is EXECUTOR.
|
||||
if outputPathPodSpecPatch == "" {
|
||||
glog.Fatalf("%s is empty. Required when driver type is %s.", argumentOutputPathPodSpecPatch, driverTypeExecutor)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,505 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/golang/glog"
|
||||
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
pb "github.com/kubeflow/pipelines/api/v2alpha1/go"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/common"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/common/mlmd"
|
||||
mlmdPb "github.com/kubeflow/pipelines/third_party/ml-metadata/go_client/ml_metadata/proto"
|
||||
"github.com/pkg/errors"
|
||||
"google.golang.org/grpc"
|
||||
k8sv1 "k8s.io/api/core/v1"
|
||||
)
|
||||
|
||||
const (
|
||||
kfpV2ExecutionType = "pipelines.kubeflow.org/v2alpha1/task"
|
||||
kfpV2ContextTypeDag = "pipelines.kubeflow.org/v2alpha1/dag"
|
||||
)
|
||||
|
||||
// execution custom parameters
|
||||
const (
|
||||
executionParamPrefixInputProperty = "input:"
|
||||
executionParamContextName = "contextName"
|
||||
// (context name, task name) is a unique key to identify a KFP execution.
|
||||
// task name is unique among the context DAG.
|
||||
executionParamTaskName = "taskName"
|
||||
)
|
||||
|
||||
// context custom parameters
|
||||
const (
|
||||
contextParamDagExecutionId = "dagExecutionId"
|
||||
)
|
||||
|
||||
func main() {
|
||||
initFlags()
|
||||
validateFlagsOrFatal()
|
||||
|
||||
taskSpec, err := unmarshalTaskSpec(taskSpecJson)
|
||||
if err != nil {
|
||||
glog.Fatalln(err)
|
||||
}
|
||||
glog.Infof("taskSpec: %s", taskSpec.String())
|
||||
var executorSpec *pb.PipelineDeploymentConfig_ExecutorSpec
|
||||
if executorSpecJson != "" {
|
||||
executorSpec, err = unmarshalExecutorSpec(executorSpecJson)
|
||||
if err != nil {
|
||||
glog.Fatalln(err)
|
||||
}
|
||||
glog.Infof("executorSpec: %s", executorSpec.String())
|
||||
}
|
||||
|
||||
// Initialize MLMD connection, client and context
|
||||
conn, err := grpc.Dial(mlmdUrl, grpc.WithInsecure(), grpc.WithBlock())
|
||||
if err != nil {
|
||||
glog.Fatalf("did not connect: %v", err)
|
||||
}
|
||||
defer conn.Close()
|
||||
c := mlmdPb.NewMetadataStoreServiceClient(conn)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
defer cancel()
|
||||
execution, err := putExecutionAndContext(ctx, c, taskSpec, driverType == driverTypeDag /** createContext*/)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
if driverType == driverTypeExecutor {
|
||||
err = driveExecutor(executorSpec, taskSpec.GetOutputs(), execution, outputPathPodSpecPatch)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
}
|
||||
glog.Flush()
|
||||
}
|
||||
|
||||
func putExecutionAndContext(
|
||||
ctx context.Context,
|
||||
c mlmdPb.MetadataStoreServiceClient,
|
||||
taskSpec *pb.PipelineTaskSpec,
|
||||
createContext bool,
|
||||
) (*mlmdPb.Execution, error) {
|
||||
// put execution type
|
||||
executionTypeName := kfpV2ExecutionType
|
||||
putExecutionTypeResponse, err := c.PutExecutionType(ctx,
|
||||
&mlmdPb.PutExecutionTypeRequest{
|
||||
ExecutionType: &mlmdPb.ExecutionType{
|
||||
Name: &executionTypeName,
|
||||
},
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
executionTypeId := putExecutionTypeResponse.GetTypeId()
|
||||
if executionTypeId == 0 {
|
||||
return nil, errors.Errorf("Execution type ID is 0, %v", putExecutionTypeResponse)
|
||||
}
|
||||
|
||||
// put context type
|
||||
contextTypeName := kfpV2ContextTypeDag
|
||||
putContextTypeResponse, err := c.PutContextType(ctx,
|
||||
&mlmdPb.PutContextTypeRequest{
|
||||
ContextType: &mlmdPb.ContextType{
|
||||
Name: &contextTypeName,
|
||||
},
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
contextTypeId := putContextTypeResponse.GetTypeId()
|
||||
if contextTypeId == 0 {
|
||||
return nil, errors.Errorf("Context type ID is 0, %v", putContextTypeResponse)
|
||||
}
|
||||
|
||||
// put execution
|
||||
contextName := ""
|
||||
if createContext {
|
||||
contextName = executionName
|
||||
}
|
||||
var parentContext *mlmdPb.Context
|
||||
if parentContextName != "" {
|
||||
// get parent context
|
||||
contextTypeName := kfpV2ContextTypeDag
|
||||
parentContextResponse, err := c.GetContextByTypeAndName(ctx, &mlmdPb.GetContextByTypeAndNameRequest{
|
||||
TypeName: &contextTypeName,
|
||||
ContextName: &parentContextName,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
parentContext = parentContextResponse.GetContext()
|
||||
if parentContext == nil {
|
||||
return nil, errors.Errorf("parentContext is nil. response: %s", parentContextResponse.String())
|
||||
}
|
||||
}
|
||||
execution, err := initExecution(ctx, c, executionName, executionTypeId, taskSpec, contextName, parentContext, outputPathParameters)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to init execution")
|
||||
}
|
||||
glog.Infof("initExecution: %s\n", execution.String())
|
||||
var contexts []*mlmdPb.Context
|
||||
if parentContext != nil {
|
||||
contexts = []*mlmdPb.Context{
|
||||
parentContext,
|
||||
}
|
||||
}
|
||||
putExecutionResponse, err := c.PutExecution(
|
||||
ctx,
|
||||
&mlmdPb.PutExecutionRequest{
|
||||
Execution: execution,
|
||||
Contexts: contexts,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to put execution")
|
||||
}
|
||||
executionId := putExecutionResponse.GetExecutionId()
|
||||
if executionId == 0 {
|
||||
return nil, errors.Errorf("Failed to put execution, execution ID is 0: %v", putExecutionResponse.String())
|
||||
}
|
||||
glog.Infof("PutExecutionResponse: %s\n", putExecutionResponse.String())
|
||||
execution.Id = &executionId
|
||||
|
||||
err = os.MkdirAll(filepath.Dir(outputPathExecutionId), os.ModePerm)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
// TODO(Bobgy): figure out an explanation of 0644
|
||||
err = ioutil.WriteFile(outputPathExecutionId, []byte(fmt.Sprint(executionId)), 0644)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
|
||||
if createContext {
|
||||
// put context
|
||||
putContextResponse, err := c.PutContexts(ctx, &mlmdPb.PutContextsRequest{
|
||||
Contexts: []*mlmdPb.Context{
|
||||
{
|
||||
Name: &contextName,
|
||||
TypeId: &contextTypeId,
|
||||
CustomProperties: map[string]*mlmdPb.Value{
|
||||
contextParamDagExecutionId: {
|
||||
Value: &mlmdPb.Value_IntValue{
|
||||
IntValue: executionId,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
})
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to put context")
|
||||
}
|
||||
contextIds := putContextResponse.GetContextIds()
|
||||
if len(contextIds) != 1 {
|
||||
return nil, errors.Errorf("Failed to put context: unexpected length of context IDs in put context response: %v", contextIds)
|
||||
}
|
||||
contextId := contextIds[0]
|
||||
if contextId == 0 {
|
||||
return nil, errors.Errorf("Failed to put context: context ID in response is 0.")
|
||||
}
|
||||
glog.Infof("PutContextResponse: %s\n", putContextResponse.String())
|
||||
|
||||
// Example: we can find this context like this.
|
||||
// foundContext, err := c.GetContextByTypeAndName(ctx, &mlmdPb.GetContextByTypeAndNameRequest{
|
||||
// TypeName: &contextTypeName,
|
||||
// ContextName: &contextName,
|
||||
// })
|
||||
// glog.Infof("%v", foundContext.String())
|
||||
|
||||
err = os.MkdirAll(filepath.Dir(outputPathContextName), os.ModePerm)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
err = ioutil.WriteFile(outputPathContextName, []byte(contextName), 0644)
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
return execution, nil
|
||||
}
|
||||
|
||||
func initExecution(
|
||||
ctx context.Context,
|
||||
c mlmdPb.MetadataStoreServiceClient,
|
||||
name string, typeId int64, taskSpec *pb.PipelineTaskSpec, contextName string,
|
||||
parentContext *mlmdPb.Context,
|
||||
outputPathParameters string,
|
||||
) (*mlmdPb.Execution, error) {
|
||||
var parentExecution *mlmdPb.Execution
|
||||
executionByTaskName := make(map[string]*mlmdPb.Execution)
|
||||
|
||||
// There is only one case that parent context does not exist: root DAG.
|
||||
if parentContext != nil {
|
||||
// get parent execution
|
||||
parentExecutionId := parentContext.CustomProperties[contextParamDagExecutionId].GetIntValue()
|
||||
if parentExecutionId == 0 {
|
||||
return nil, errors.Errorf("Cannot get parent execution ID from parent context: %v", parentContext.String())
|
||||
}
|
||||
parentExecutionResponse, err := c.GetExecutionsByID(ctx, &mlmdPb.GetExecutionsByIDRequest{
|
||||
ExecutionIds: []int64{parentExecutionId},
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(parentExecutionResponse.GetExecutions()) != 1 {
|
||||
return nil, errors.Errorf("Unexpected number of executions in get executions by ID response: (ID=%v response: %v)", parentExecutionId, parentExecutionResponse)
|
||||
}
|
||||
parentExecution = parentExecutionResponse.GetExecutions()[0]
|
||||
if parentExecution == nil {
|
||||
return nil, errors.Errorf("Parent execution is nil: (ID=%v response: %v)", parentExecutionId, parentExecutionResponse)
|
||||
}
|
||||
glog.Infof("Parent execution: %v", parentExecution.String())
|
||||
|
||||
// get all executions in the same context
|
||||
executionsResponse, err := c.GetExecutionsByContext(ctx, &mlmdPb.GetExecutionsByContextRequest{
|
||||
ContextId: parentContext.Id,
|
||||
Options: &mlmdPb.ListOperationOptions{},
|
||||
})
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed getting executions by context: context_name=%v, context_id=%v", parentContextName, parentContext.Id)
|
||||
}
|
||||
// TODO(Bobgy): support pagination and more executions in the same context
|
||||
if executionsResponse.GetNextPageToken() != "" {
|
||||
return nil, errors.Errorf("Too many (>%v) executions in the same context, when getting executions by context: context_name=%v, context_id=%v.", len(executionsResponse.GetExecutions()), parentContextName, parentContext.Id)
|
||||
}
|
||||
executions := executionsResponse.GetExecutions()
|
||||
|
||||
// Convert execution list to a map keyed by DAG task name.
|
||||
for _, execution := range executions {
|
||||
customProperties := execution.GetCustomProperties()
|
||||
if customProperties == nil {
|
||||
glog.Warningf("Execution id=%v name=%v does not have custom properties", execution.GetId(), execution.GetName())
|
||||
continue
|
||||
}
|
||||
taskNameProp := customProperties[executionParamTaskName]
|
||||
if taskNameProp == nil {
|
||||
glog.Warningf("Execution id=%v name=%v does not have %s custom property", execution.GetId(), execution.GetName(), executionParamTaskName)
|
||||
continue
|
||||
}
|
||||
switch value := taskNameProp.Value.(type) {
|
||||
case *mlmdPb.Value_StringValue:
|
||||
executionByTaskName[value.StringValue] = execution
|
||||
default:
|
||||
glog.Warningf("Unexpected execution task name type: expected string, but got unrecognized type: %T.", taskNameProp.Value)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// initialize execution
|
||||
var execution mlmdPb.Execution
|
||||
execution.Name = &name
|
||||
execution.TypeId = &typeId
|
||||
state := mlmdPb.Execution_RUNNING
|
||||
execution.LastKnownState = &state
|
||||
execution.CustomProperties = make(map[string]*mlmdPb.Value)
|
||||
taskName := taskSpec.GetTaskInfo().GetName()
|
||||
if taskName != "" {
|
||||
execution.CustomProperties[executionParamTaskName] = &mlmdPb.Value{
|
||||
Value: &mlmdPb.Value_StringValue{
|
||||
StringValue: taskName,
|
||||
},
|
||||
}
|
||||
}
|
||||
if contextName != "" {
|
||||
// Note, the context can be queried via GetContextByTypeAndName.
|
||||
execution.CustomProperties[executionParamContextName] =
|
||||
&mlmdPb.Value{Value: &mlmdPb.Value_StringValue{StringValue: contextName}}
|
||||
}
|
||||
// resolve parameters
|
||||
if outputPathParameters != "" {
|
||||
// When driving a task, we need to expose parameters as argo parameters.
|
||||
err := os.MkdirAll(outputPathParameters, os.ModePerm)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to make all parent dirs for -%s=%v", argumentOutputPathParameters, outputPathParameters)
|
||||
}
|
||||
}
|
||||
for name, parameter := range taskSpec.GetInputs().GetParameters() {
|
||||
// TODO(Bobgy): handle other parameter types
|
||||
|
||||
// resolve parameter value
|
||||
var value string
|
||||
|
||||
if parameter.GetRuntimeValue() != nil {
|
||||
runtimeValue := parameter.GetRuntimeValue()
|
||||
if runtimeValue.GetConstantValue() != nil {
|
||||
// TODO(Bobgy): build conversion helpers between MLMD Value and KFP Pipeline Spec Value, and use them here.
|
||||
value = runtimeValue.GetConstantValue().GetStringValue()
|
||||
execution.CustomProperties[executionParamPrefixInputProperty+name] =
|
||||
&mlmdPb.Value{Value: &mlmdPb.Value_StringValue{
|
||||
StringValue: value}}
|
||||
} else {
|
||||
return nil, errors.Errorf("Unsupported parameter type: %T", parameter.GetRuntimeValue().Value)
|
||||
}
|
||||
} else if parameter.GetTaskOutputParameter() != nil {
|
||||
spec := parameter.GetTaskOutputParameter()
|
||||
if spec.ProducerTask == "" {
|
||||
return nil, errors.Errorf("Task output parameter's producer task is empty: %v", parameter.String())
|
||||
}
|
||||
producerExecution := executionByTaskName[spec.ProducerTask]
|
||||
if producerExecution == nil {
|
||||
return nil, errors.Errorf("Cannot find producer execution for parameter '%v' in parent context with name '%v'.", parameter.String(), parentContextName)
|
||||
}
|
||||
producerKfpExecution := mlmd.NewKfpExecution(producerExecution)
|
||||
if spec.GetOutputParameterKey() == "" {
|
||||
return nil, errors.Errorf("Output parameter key is empty for parameter '%v'.", parameter.String())
|
||||
}
|
||||
value := producerKfpExecution.GetOutputParameter(spec.GetOutputParameterKey())
|
||||
// All parameters are required for KFP.
|
||||
if value == nil {
|
||||
return nil, errors.Errorf("Producer execution does not have parameter '%v'. Producer: %v", parameter.String(), producerKfpExecution.String())
|
||||
}
|
||||
execution.CustomProperties[executionParamPrefixInputProperty+name] = value
|
||||
// TODO(Bobgy): type check using component inputs spec.
|
||||
} else if parameter.GetComponentInputParameter() != "" {
|
||||
if parentExecution == nil || parentContext == nil {
|
||||
return nil, errors.Errorf("ComponentInputParameter should not be specified in root DAG.")
|
||||
}
|
||||
customProperties := parentExecution.GetCustomProperties()
|
||||
if customProperties == nil {
|
||||
return nil, errors.Errorf("ComponentInputParameter '%s' specified, but parent execution does not have any parameters", parameter.GetComponentInputParameter())
|
||||
}
|
||||
parentInputPropertyName := executionParamPrefixInputProperty + parameter.GetComponentInputParameter()
|
||||
parentInputProperty := customProperties[parentInputPropertyName]
|
||||
if parentInputProperty == nil {
|
||||
return nil, errors.Errorf("ComponentInputParameter '%s' specified, but parent execution does not have '%s' custom property", parameter.GetComponentInputParameter(), parentInputPropertyName)
|
||||
}
|
||||
value = parentInputProperty.GetStringValue()
|
||||
// TODO(Bobgy): verify custom property type matches and support other types
|
||||
execution.CustomProperties[executionParamPrefixInputProperty+name] =
|
||||
&mlmdPb.Value{Value: &mlmdPb.Value_StringValue{
|
||||
StringValue: value}}
|
||||
}
|
||||
|
||||
if outputPathParameters != "" {
|
||||
// write parameter value to output path
|
||||
filePath := filepath.Join(outputPathParameters, name)
|
||||
err := ioutil.WriteFile(filePath, []byte(value), 0644)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to write parameter '%s' to file '%s'", name, filePath)
|
||||
}
|
||||
}
|
||||
}
|
||||
return &execution, nil
|
||||
}
|
||||
|
||||
func driveExecutor(
|
||||
executorSpec *pb.PipelineDeploymentConfig_ExecutorSpec,
|
||||
outputsSpec *pb.TaskOutputsSpec,
|
||||
execution *mlmdPb.Execution,
|
||||
outputPathPodSpecPatch string,
|
||||
) error {
|
||||
var podSpec k8sv1.PodSpec
|
||||
podSpec.Containers = []k8sv1.Container{
|
||||
{
|
||||
Name: "main",
|
||||
Image: executorSpec.GetContainer().GetImage(),
|
||||
},
|
||||
}
|
||||
mainContainer := &podSpec.Containers[0]
|
||||
|
||||
mainContainer.Image = fillPlaceholders(mainContainer.Image, execution, outputsSpec)
|
||||
marshaler := jsonpb.Marshaler{}
|
||||
if outputsSpec == nil {
|
||||
outputsSpec = &pb.TaskOutputsSpec{}
|
||||
}
|
||||
outputsSpecJson, err := marshaler.MarshalToString(outputsSpec)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "Failed to marshal outputs spec")
|
||||
}
|
||||
mainContainer.Command = []string{
|
||||
common.ExecutorEntrypointPath,
|
||||
"--logtostderr",
|
||||
"--mlmd_url=metadata-grpc-service.kubeflow.svc.cluster.local:8080",
|
||||
fmt.Sprintf("--component_outputs_spec=%s", outputsSpecJson),
|
||||
fmt.Sprintf("--execution_id=%v", execution.GetId()),
|
||||
"--publisher_type=EXECUTOR",
|
||||
"--input_path_parameters=" + common.ExecutorOutputPathParameters,
|
||||
"--", // append "--", so that all the following arguments will not be parsed as flags.
|
||||
}
|
||||
mainContainer.Args = append(executorSpec.GetContainer().GetCommand(), executorSpec.GetContainer().GetArgs()...)
|
||||
fillPlaceholdersForArray(&mainContainer.Args, execution, outputsSpec)
|
||||
|
||||
// write pod spec patch to output path
|
||||
podSpecPatchJsonBytes, err := json.Marshal(podSpec)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "Failed to marshal pod spec to JSON. PodSpec: %v", podSpec)
|
||||
}
|
||||
err = ioutil.WriteFile(outputPathPodSpecPatch, podSpecPatchJsonBytes, 0644)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "Failed to pod spec patch to file '%s'", outputPathPodSpecPatch)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func fillPlaceholdersForArray(s *[]string, execution *mlmdPb.Execution, outputsSpec *pb.TaskOutputsSpec) {
|
||||
for index, item := range *s {
|
||||
(*s)[index] = fillPlaceholders(item, execution, outputsSpec)
|
||||
}
|
||||
}
|
||||
|
||||
func fillPlaceholders(s string, execution *mlmdPb.Execution, outputsSpec *pb.TaskOutputsSpec) string {
|
||||
properties := &execution.CustomProperties
|
||||
if properties != nil {
|
||||
for propertyName, property := range *properties {
|
||||
if strings.HasPrefix(propertyName, executionParamPrefixInputProperty) {
|
||||
name := strings.TrimPrefix(propertyName, executionParamPrefixInputProperty)
|
||||
// TODO(Bobgy): handle other value types
|
||||
value := property.GetStringValue()
|
||||
// TODO(Bobgy): figure out a more stable way to replace placeholders
|
||||
s = strings.ReplaceAll(s, fmt.Sprintf("{{$.inputs.parameters['%s']}}", name), value)
|
||||
}
|
||||
}
|
||||
}
|
||||
parameters := outputsSpec.GetParameters()
|
||||
if parameters != nil {
|
||||
for name := range parameters {
|
||||
s = strings.ReplaceAll(
|
||||
s,
|
||||
fmt.Sprintf("{{$.outputs.parameters['%s'].output_file}}", name),
|
||||
filepath.Join(common.ExecutorOutputPathParameters, name),
|
||||
)
|
||||
}
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func unmarshalTaskSpec(json string) (*pb.PipelineTaskSpec, error) {
|
||||
taskSpec := &pb.PipelineTaskSpec{}
|
||||
if err := jsonpb.UnmarshalString(json, taskSpec); err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to parse task spec: %v", json)
|
||||
}
|
||||
return taskSpec, nil
|
||||
}
|
||||
|
||||
func unmarshalExecutorSpec(json string) (*pb.PipelineDeploymentConfig_ExecutorSpec, error) {
|
||||
executorSpec := &pb.PipelineDeploymentConfig_ExecutorSpec{}
|
||||
if err := jsonpb.UnmarshalString(json, executorSpec); err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to parse executor spec: %v", json)
|
||||
}
|
||||
return executorSpec, nil
|
||||
}
|
||||
|
|
@ -0,0 +1,3 @@
|
|||
FROM alpine
|
||||
WORKDIR /bin
|
||||
COPY main /bin/kfp-entrypoint
|
||||
|
|
@ -0,0 +1,54 @@
|
|||
REPO_ROOT=../../../..
|
||||
GO_COMMON_ARGS=--logtostderr
|
||||
IMAGE=kfp-entrypoint
|
||||
EXECUTION_ID=160
|
||||
PUBLISHER_EXAMPLES=../publisher/example
|
||||
|
||||
# Example .env file:
|
||||
# ```
|
||||
# IMAGE_DEV=gcr.io/gongyuan-pipeline-test/kfp-publisher
|
||||
# ```
|
||||
# It should be an image repository where you can push to via docker push.
|
||||
include .env
|
||||
|
||||
.PHONY: run
|
||||
run: *.go
|
||||
# Make sure you have a separate running terminal that runs
|
||||
# `make proxy`.
|
||||
|
||||
# Clean up intermediate data
|
||||
rm -rf ./data
|
||||
# Run executor publisher
|
||||
go run *.go $(GO_COMMON_ARGS) \
|
||||
--component_outputs_spec="$$(cat $(PUBLISHER_EXAMPLES)/outputs_spec.json)" \
|
||||
--execution_id=$(EXECUTION_ID) \
|
||||
--publisher_type=EXECUTOR \
|
||||
--input_path_parameters=$(PUBLISHER_EXAMPLES)/parameters \
|
||||
-- echo -n Hello, World!
|
||||
|
||||
.PHONY: run-docker
|
||||
run-docker: .build
|
||||
docker run $(IMAGE) /bin/kfp-entrypoint $(GO_COMMON_ARGS) \
|
||||
--component_outputs_spec="$$(cat $(PUBLISHER_EXAMPLES)/outputs_spec.json)" \
|
||||
--execution_id=$(EXECUTION_ID) \
|
||||
--publisher_type=EXECUTOR \
|
||||
--input_path_parameters=$(PUBLISHER_EXAMPLES)/parameters \
|
||||
-- echo -n Hello, World!
|
||||
|
||||
.PHONY: proxy
|
||||
proxy:
|
||||
kubectl port-forward -n kubeflow svc/metadata-grpc-service 8080:8080
|
||||
|
||||
.PHONY: image
|
||||
image: .build
|
||||
|
||||
.PHONY: image-push
|
||||
image-push: image
|
||||
docker push $(IMAGE_DEV)
|
||||
|
||||
.build: Dockerfile.dev main .env
|
||||
docker build . -f Dockerfile.dev -t $(IMAGE) -t $(IMAGE_DEV)
|
||||
touch .build
|
||||
|
||||
main: *.go $(REPO_ROOT)/go.mod
|
||||
env GOOS=linux CGO_ENABLED=0 go build -ldflags '-extldflags "-static"' -o main *.go
|
||||
|
|
@ -0,0 +1,41 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"os"
|
||||
"os/exec"
|
||||
|
||||
"github.com/golang/glog"
|
||||
publisherCmd "github.com/kubeflow/pipelines/backend/src/v2/publisher/cmd"
|
||||
)
|
||||
|
||||
func main() {
|
||||
publisherCmd.InitFlags()
|
||||
publisherCmd.ValidateFlagsOrFatal()
|
||||
glog.Infof("Command and arguments: %v", flag.Args())
|
||||
cmd := exec.Command(flag.Args()[0], flag.Args()[1:]...)
|
||||
cmd.Stdin = os.Stdin
|
||||
cmd.Stdout = os.Stdout
|
||||
cmd.Stderr = os.Stderr
|
||||
if err := cmd.Run(); err != nil {
|
||||
publisherCmd.Publish()
|
||||
glog.Fatal(err)
|
||||
} else {
|
||||
publisherCmd.Publish()
|
||||
}
|
||||
glog.Flush()
|
||||
}
|
||||
|
|
@ -0,0 +1,4 @@
|
|||
FROM scratch
|
||||
WORKDIR /bin
|
||||
COPY main /bin/kfp-publisher
|
||||
ENTRYPOINT ["/bin/kfp-publisher"]
|
||||
|
|
@ -0,0 +1,51 @@
|
|||
REPO_ROOT=../../../..
|
||||
GO_COMMON_ARGS=--logtostderr
|
||||
EXECUTION_ID=160
|
||||
IMAGE=kfp-publisher
|
||||
|
||||
# Example .env file:
|
||||
# ```
|
||||
# IMAGE_DEV=gcr.io/gongyuan-pipeline-test/kfp-publisher
|
||||
# ```
|
||||
# It should be an image repository where you can push to via docker push.
|
||||
include .env
|
||||
|
||||
.PHONY: run
|
||||
run: *.go example/outputs_spec.json
|
||||
# Make sure you have a separate running terminal that runs
|
||||
# `make proxy`.
|
||||
|
||||
# Clean up intermediate data
|
||||
rm -rf ./data
|
||||
# Run executor publisher
|
||||
go run *.go $(GO_COMMON_ARGS) \
|
||||
--component_outputs_spec="$$(cat ./example/outputs_spec.json)" \
|
||||
--execution_id=$(EXECUTION_ID) \
|
||||
--publisher_type=EXECUTOR \
|
||||
--input_path_parameters=./example/parameters
|
||||
|
||||
.PHONY: run-docker
|
||||
run-docker: .build
|
||||
docker run kfp/publisher $(GO_COMMON_ARGS) \
|
||||
--component_outputs_spec="$$(cat ./example/outputs_spec.json)" \
|
||||
--execution_id=$(EXECUTION_ID) \
|
||||
--publisher_type=EXECUTOR \
|
||||
--input_path_parameters=./example/parameters
|
||||
|
||||
.PHONY: proxy
|
||||
proxy:
|
||||
kubectl port-forward -n kubeflow svc/metadata-grpc-service 8080:8080
|
||||
|
||||
.PHONY: image
|
||||
image: .build
|
||||
|
||||
.PHONY: image-push
|
||||
image-push: image
|
||||
docker push $(IMAGE_DEV)
|
||||
|
||||
.build: Dockerfile.dev main .env
|
||||
docker build . -f Dockerfile.dev -t $(IMAGE) -t $(IMAGE_DEV)
|
||||
touch .build
|
||||
|
||||
main: *.go $(REPO_ROOT)/go.mod
|
||||
env GOOS=linux CGO_ENABLED=0 go build -ldflags '-extldflags "-static"' -o main *.go
|
||||
|
|
@ -0,0 +1,71 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/golang/glog"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/common"
|
||||
)
|
||||
|
||||
// command line arguments
|
||||
const (
|
||||
argumentMlmdUrl = "mlmd_url"
|
||||
argumentMlmdUrlDefault = "localhost:8080"
|
||||
argumentComponentOutputsSpec = "component_outputs_spec"
|
||||
argumentExecutionId = "execution_id"
|
||||
argumentPublisherType = "publisher_type"
|
||||
argumentInputPathParameters = "input_path_parameters"
|
||||
)
|
||||
|
||||
// command line variables
|
||||
var (
|
||||
mlmdUrl string
|
||||
componentOutputsSpecJson string
|
||||
executionId int64
|
||||
publisherType string
|
||||
inputPathParameters string
|
||||
)
|
||||
|
||||
func InitFlags() {
|
||||
flag.StringVar(&mlmdUrl, argumentMlmdUrl, argumentMlmdUrlDefault, "URL of MLMD, defaults to localhost:8080")
|
||||
flag.StringVar(&componentOutputsSpecJson, argumentComponentOutputsSpec, "", "Component outputs spec")
|
||||
flag.Int64Var(&executionId, argumentExecutionId, 0, "Execution ID to publish")
|
||||
flag.StringVar(&publisherType, argumentPublisherType, "", fmt.Sprintf("Publisher type, can be '%s' or '%s'", common.PublisherType_DAG, common.PublisherType_EXECUTOR))
|
||||
flag.StringVar(&inputPathParameters, argumentInputPathParameters, "", "Input path which contains files corresponding to parameter values")
|
||||
|
||||
flag.Parse()
|
||||
glog.Infof("Publisher arguments: %v", os.Args)
|
||||
}
|
||||
|
||||
func ValidateFlagsOrFatal() {
|
||||
if publisherType == "" {
|
||||
glog.Fatalln(argumentPublisherType + " is not provided.")
|
||||
} else if publisherType != common.PublisherType_DAG && publisherType != common.PublisherType_EXECUTOR {
|
||||
glog.Fatalf("invalid %s provided: %s. It should be either '%s' or '%s'", argumentPublisherType, publisherType, common.PublisherType_DAG, common.PublisherType_EXECUTOR)
|
||||
}
|
||||
if executionId == 0 {
|
||||
glog.Fatalln(argumentExecutionId + " is not provided.")
|
||||
}
|
||||
if componentOutputsSpecJson == "" {
|
||||
glog.Fatalln(argumentComponentOutputsSpec + " is not provided.")
|
||||
}
|
||||
if inputPathParameters == "" {
|
||||
glog.Fatalln(argumentInputPathParameters + " is not provided.")
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,130 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io/ioutil"
|
||||
"path/filepath"
|
||||
"time"
|
||||
|
||||
"github.com/golang/glog"
|
||||
"github.com/golang/protobuf/jsonpb"
|
||||
pb "github.com/kubeflow/pipelines/api/v2alpha1/go"
|
||||
mlmdPb "github.com/kubeflow/pipelines/third_party/ml-metadata/go_client/ml_metadata/proto"
|
||||
"github.com/pkg/errors"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
const (
|
||||
executionParamPrefixOutputProperty = "output:"
|
||||
)
|
||||
|
||||
type publishArgs struct {
|
||||
outputsSpec *pb.ComponentOutputsSpec
|
||||
executionId int64
|
||||
mlmdUrl string
|
||||
publisherType string
|
||||
inputPathParameters string
|
||||
}
|
||||
|
||||
type mlmdClientHelper struct {
|
||||
client mlmdPb.MetadataStoreServiceClient
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
// TODO(Bobgy): refactor driver to also use this client
|
||||
func (c *mlmdClientHelper) getExecutionByID(id int64) (*mlmdPb.Execution, error) {
|
||||
response, err := c.client.GetExecutionsByID(c.ctx, &mlmdPb.GetExecutionsByIDRequest{
|
||||
ExecutionIds: []int64{id},
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(response.Executions) != 1 {
|
||||
return nil, errors.Errorf("Unexpected number of executions returned: '%v',when fetching execution with ID %v", len(response.Executions), id)
|
||||
}
|
||||
return response.Executions[0], nil
|
||||
}
|
||||
|
||||
func publishImp(args *publishArgs) error {
|
||||
// Initialize MLMD connection, client and context
|
||||
conn, err := grpc.Dial(mlmdUrl, grpc.WithInsecure(), grpc.WithBlock())
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "did not connect")
|
||||
}
|
||||
defer conn.Close()
|
||||
c := mlmdPb.NewMetadataStoreServiceClient(conn)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
defer cancel()
|
||||
|
||||
mlmdClient := mlmdClientHelper{client: c, ctx: ctx}
|
||||
|
||||
execution, err := mlmdClient.getExecutionByID(args.executionId)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "Failed to get execution.")
|
||||
}
|
||||
glog.Infof("Execution before update: %v", execution.String())
|
||||
execution.Id = &args.executionId
|
||||
if execution.CustomProperties == nil {
|
||||
execution.CustomProperties = make(map[string]*mlmdPb.Value)
|
||||
}
|
||||
for name := range args.outputsSpec.Parameters {
|
||||
content, err := ioutil.ReadFile(filepath.Join(inputPathParameters, name))
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "Failed to read content of parameter %s.", name)
|
||||
}
|
||||
// TODO(Bobgy): support other types and validate parameter type
|
||||
execution.CustomProperties[executionParamPrefixOutputProperty+name] = &mlmdPb.Value{
|
||||
Value: &mlmdPb.Value_StringValue{
|
||||
StringValue: string(content),
|
||||
},
|
||||
}
|
||||
}
|
||||
state := mlmdPb.Execution_COMPLETE
|
||||
execution.LastKnownState = &state
|
||||
_, err = c.PutExecution(ctx, &mlmdPb.PutExecutionRequest{Execution: execution})
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "Failed to put execution")
|
||||
}
|
||||
execution2, err := mlmdClient.getExecutionByID(args.executionId)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "Failed to get execution.")
|
||||
}
|
||||
glog.Infof("Execution after update: %v", execution2.String())
|
||||
return nil
|
||||
}
|
||||
|
||||
func unmarshalOutputsSpec(json string) (*pb.ComponentOutputsSpec, error) {
|
||||
outputsSpec := &pb.ComponentOutputsSpec{}
|
||||
if err := jsonpb.UnmarshalString(json, outputsSpec); err != nil {
|
||||
return nil, errors.Wrapf(err, "Failed to parse component outputs spec: %v", json)
|
||||
}
|
||||
return outputsSpec, nil
|
||||
}
|
||||
|
||||
func Publish() error {
|
||||
outputsSpec, err := unmarshalOutputsSpec(componentOutputsSpecJson)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return publishImp(&publishArgs{
|
||||
outputsSpec: outputsSpec,
|
||||
executionId: executionId,
|
||||
mlmdUrl: mlmdUrl,
|
||||
publisherType: publisherType,
|
||||
inputPathParameters: inputPathParameters,
|
||||
})
|
||||
}
|
||||
|
|
@ -0,0 +1,14 @@
|
|||
{
|
||||
"parameters": {
|
||||
"output_value": {
|
||||
"type": "STRING"
|
||||
}
|
||||
},
|
||||
"artifacts": {
|
||||
"output_artifact": {
|
||||
"artifactType": {
|
||||
"instanceSchema": "properties:\ntitle: kfp.Artifact\ntype: object\n"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1 @@
|
|||
Hello, World!
|
||||
|
|
@ -0,0 +1,30 @@
|
|||
// Copyright 2021 Google LLC
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"github.com/golang/glog"
|
||||
"github.com/kubeflow/pipelines/backend/src/v2/publisher/cmd"
|
||||
)
|
||||
|
||||
func main() {
|
||||
cmd.InitFlags()
|
||||
cmd.ValidateFlagsOrFatal()
|
||||
err := cmd.Publish()
|
||||
if err != nil {
|
||||
glog.Fatal(err)
|
||||
}
|
||||
glog.Flush()
|
||||
}
|
||||
8
go.mod
8
go.mod
|
|
@ -20,7 +20,7 @@ require (
|
|||
github.com/golang/protobuf v1.4.2
|
||||
github.com/google/addlicense v0.0.0-20200906110928-a0294312aa76 // indirect
|
||||
github.com/google/go-cmp v0.5.2
|
||||
github.com/google/uuid v1.1.1
|
||||
github.com/google/uuid v1.1.2
|
||||
github.com/gopherjs/gopherjs v0.0.0-20181103185306-d547d1d9531e // indirect
|
||||
github.com/gorilla/mux v1.8.0
|
||||
github.com/grpc-ecosystem/grpc-gateway v1.12.1
|
||||
|
|
@ -40,8 +40,10 @@ require (
|
|||
golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7
|
||||
golang.org/x/sync v0.0.0-20201207232520-09787c993a3a // indirect
|
||||
google.golang.org/api v0.20.0
|
||||
google.golang.org/genproto v0.0.0-20200317114155-1f3552e48f24
|
||||
google.golang.org/grpc v1.28.0
|
||||
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013
|
||||
google.golang.org/grpc v1.34.0
|
||||
google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.0.1 // indirect
|
||||
google.golang.org/protobuf v1.25.0
|
||||
gopkg.in/yaml.v2 v2.3.0
|
||||
k8s.io/api v0.17.9
|
||||
k8s.io/apimachinery v0.17.9
|
||||
|
|
|
|||
|
|
@ -83,6 +83,7 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P
|
|||
github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU=
|
||||
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
|
||||
github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc=
|
||||
github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk=
|
||||
github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
|
||||
github.com/colinmarc/hdfs v1.1.4-0.20180805212432-9746310a4d31/go.mod h1:vSBumefK4HA5uiRSwNP+3ofgrEoScpCS2MMWcWXEuQ4=
|
||||
github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
|
||||
|
|
@ -124,6 +125,7 @@ github.com/emirpasic/gods v1.12.0/go.mod h1:YfzfFFoVP/catgzJb4IKIqXjX78Ha8FMSDh3
|
|||
github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
|
||||
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
|
||||
github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98=
|
||||
github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po=
|
||||
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
|
||||
github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5 h1:Yzb9+7DPaBjB8zlTR87/ElzFsnQfuHnVUVqpZZIcV5Y=
|
||||
github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a2zkGnVExMxdzMo3M0Hi/3sEU+cWnZpSni0O6/Yb/P0=
|
||||
|
|
@ -263,6 +265,7 @@ github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:x
|
|||
github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
|
||||
github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
|
||||
github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
|
||||
github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8=
|
||||
github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0=
|
||||
github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
|
||||
github.com/google/addlicense v0.0.0-20200906110928-a0294312aa76 h1:JypWNzPMSgH5yL0NvFoAIsDRlKFgL0AsS3GO5bg4Pto=
|
||||
|
|
@ -277,6 +280,7 @@ github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4=
|
|||
github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
|
||||
github.com/google/go-cmp v0.4.1 h1:/exdXoGamhu5ONeUJH0deniYLWYvQwW66yvlfiiKTu0=
|
||||
github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
|
||||
github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
|
||||
github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM=
|
||||
github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
|
||||
github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
|
||||
|
|
@ -294,6 +298,8 @@ github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm4
|
|||
github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
|
||||
github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY=
|
||||
github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
|
||||
github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y=
|
||||
github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
|
||||
github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
|
||||
github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM=
|
||||
github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk=
|
||||
|
|
@ -820,6 +826,8 @@ google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfG
|
|||
google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200317114155-1f3552e48f24 h1:IGPykv426z7LZSVPlaPufOyphngM4at5uZ7x5alaFvE=
|
||||
google.golang.org/genproto v0.0.0-20200317114155-1f3552e48f24/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
|
||||
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 h1:+kGHl1aib/qcwaRi1CbqBZ1rk19r85MNUf8HaBghugY=
|
||||
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
|
||||
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
|
||||
google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
|
||||
google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
|
||||
|
|
@ -832,13 +840,21 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8
|
|||
google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
|
||||
google.golang.org/grpc v1.28.0 h1:bO/TA4OxCOummhSf10siHuG7vJOiwh7SpRpFZDkOgl4=
|
||||
google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60=
|
||||
google.golang.org/grpc v1.34.0 h1:raiipEjMOIC/TO2AvyTxP25XFdLxNIBwzDh3FM3XztI=
|
||||
google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA51WJ8=
|
||||
google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.0.1 h1:M8spwkmx0pHrPq+uMdl22w5CvJ/Y+oAJTIs9oGoCpOE=
|
||||
google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.0.1/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw=
|
||||
google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
|
||||
google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
|
||||
google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
|
||||
google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
|
||||
google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
|
||||
google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
|
||||
google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM=
|
||||
google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
|
||||
google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
|
||||
google.golang.org/protobuf v1.25.0 h1:Ejskq+SyPohKW+1uil0JJMtmHCgJPJ/qWTxr8qp+R4c=
|
||||
google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c=
|
||||
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
|
||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
|
|
|
|||
|
|
@ -0,0 +1,4 @@
|
|||
# Sample for KFP v2 POC
|
||||
|
||||
Note, the generated `hw_pipeline_job.json` and `two_step_pipeline_job.json` spec doesn't fully conform to the
|
||||
latest updated IR spec. There were some manual editing in there to make the demo work.
|
||||
|
|
@ -0,0 +1,54 @@
|
|||
# Copyright 2021 Google LLC
|
||||
#
|
||||
# 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.v2 import dsl
|
||||
from kfp.v2 import compiler
|
||||
from kfp.v2 import components
|
||||
|
||||
|
||||
def main(pipeline_root: str = 'gs://gongyuan-test/hello_world'):
|
||||
|
||||
def hello_world(text: str):
|
||||
print(text)
|
||||
return text
|
||||
|
||||
components.func_to_container_op(
|
||||
hello_world, output_component_file='hw.yaml'
|
||||
)
|
||||
|
||||
# Create a pipeline op from the component we defined above.
|
||||
hw_op = components.load_component_from_file(
|
||||
'./hw.yaml'
|
||||
) # you can also use load_component_from_url
|
||||
|
||||
@dsl.pipeline(name='hello-world', description='A simple intro pipeline')
|
||||
def pipeline_parameter_to_consumer(text: str = 'hi there'):
|
||||
'''Pipeline that passes small pipeline parameter string to consumer op'''
|
||||
consume_task = hw_op(
|
||||
text
|
||||
) # Passing pipeline parameter as argument to consumer op
|
||||
|
||||
pipeline_func = pipeline_parameter_to_consumer
|
||||
|
||||
compiler.Compiler().compile(
|
||||
pipeline_func=pipeline_func,
|
||||
pipeline_root=pipeline_root,
|
||||
output_path='hw_pipeline_job.json'
|
||||
)
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
# execute only if run as a script
|
||||
import fire
|
||||
fire.Fire(main)
|
||||
|
|
@ -0,0 +1,38 @@
|
|||
# Copyright 2021 Google LLC
|
||||
#
|
||||
# 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.
|
||||
|
||||
name: Hello world
|
||||
inputs:
|
||||
- {name: text, type: String}
|
||||
implementation:
|
||||
container:
|
||||
image: python:3.7
|
||||
command:
|
||||
- python3
|
||||
- -u
|
||||
- -c
|
||||
- |
|
||||
def hello_world(text):
|
||||
print(text)
|
||||
return text
|
||||
|
||||
import argparse
|
||||
_parser = argparse.ArgumentParser(prog='Hello world', description='')
|
||||
_parser.add_argument("--text", dest="text", type=str, required=True, default=argparse.SUPPRESS)
|
||||
_parsed_args = vars(_parser.parse_args())
|
||||
|
||||
_outputs = hello_world(**_parsed_args)
|
||||
args:
|
||||
- --text
|
||||
- {inputValue: text}
|
||||
|
|
@ -0,0 +1,48 @@
|
|||
{
|
||||
"pipelineSpec": {
|
||||
"tasks": [
|
||||
{
|
||||
"inputs": {
|
||||
"parameters": { "text": { "componentInputParameter": "text" } }
|
||||
},
|
||||
"taskInfo": {
|
||||
"name": "Hello world"
|
||||
},
|
||||
"executorLabel": "Hello world"
|
||||
}
|
||||
],
|
||||
"schemaVersion": "v2alpha1",
|
||||
"runtimeParameters": {
|
||||
"text": {
|
||||
"type": "STRING",
|
||||
"defaultValue": {
|
||||
"stringValue": "hi there"
|
||||
}
|
||||
}
|
||||
},
|
||||
"pipelineInfo": {
|
||||
"name": "hello-world"
|
||||
},
|
||||
"deploymentConfig": {
|
||||
"@type": "type.googleapis.com/ml_pipelines.PipelineDeploymentConfig",
|
||||
"executors": {
|
||||
"Hello world": {
|
||||
"container": {
|
||||
"image": "python:3.7",
|
||||
"args": ["--text", "{{$.inputs.parameters['text']}}"],
|
||||
"command": [
|
||||
"python3",
|
||||
"-u",
|
||||
"-c",
|
||||
"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"
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"sdkVersion": "kfp-1.2.0"
|
||||
},
|
||||
"runtimeConfig": {
|
||||
"gcsOutputDirectory": "gs://gongyuan-test/hello_world"
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,72 @@
|
|||
# Copyright 2021 Google LLC
|
||||
#
|
||||
# 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.
|
||||
|
||||
PIPELINE_ROOT = 'gs://gongyuan-test/pipeline_root/'
|
||||
|
||||
# Simple two-step pipeline with 'producer' and 'consumer' steps
|
||||
from kfp.v2 import components
|
||||
from kfp.v2 import compiler
|
||||
from kfp.v2 import dsl
|
||||
|
||||
producer_op = components.load_component_from_text(
|
||||
"""
|
||||
name: Producer
|
||||
inputs:
|
||||
- {name: input_text, type: String, description: 'Represents an input parameter.'}
|
||||
outputs:
|
||||
- {name: output_value, type: String, description: 'Represents an output paramter.'}
|
||||
implementation:
|
||||
container:
|
||||
image: google/cloud-sdk:latest
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
echo "$0, this is an output parameter" | gsutil cp - "$1"
|
||||
- {inputValue: input_text}
|
||||
- {outputPath: output_value}
|
||||
"""
|
||||
)
|
||||
|
||||
consumer_op = components.load_component_from_text(
|
||||
"""
|
||||
name: Consumer
|
||||
inputs:
|
||||
- {name: input_value, type: String, description: 'Represents an input parameter. It connects to an upstream output parameter.'}
|
||||
implementation:
|
||||
container:
|
||||
image: google/cloud-sdk:latest
|
||||
command:
|
||||
- sh
|
||||
- -c
|
||||
- |
|
||||
set -e -x
|
||||
echo "Read from an input parameter: " && echo "$0"
|
||||
- {inputValue: input_value}
|
||||
"""
|
||||
)
|
||||
|
||||
|
||||
@dsl.pipeline(name='simple-two-step-pipeline')
|
||||
def two_step_pipeline(text='Hello world'):
|
||||
producer = producer_op(input_text=text)
|
||||
consumer = consumer_op(input_value=producer.outputs['output_value'])
|
||||
|
||||
|
||||
compiler.Compiler().compile(
|
||||
pipeline_func=two_step_pipeline,
|
||||
pipeline_root=PIPELINE_ROOT,
|
||||
output_path='two_step_pipeline_job.json'
|
||||
)
|
||||
|
|
@ -0,0 +1,82 @@
|
|||
{
|
||||
"pipelineSpec": {
|
||||
"sdkVersion": "kfp-1.2.0",
|
||||
"tasks": [
|
||||
{
|
||||
"inputs": {
|
||||
"parameters": { "text": { "componentInputParameter": "text" } }
|
||||
},
|
||||
"taskInfo": {
|
||||
"name": "Producer"
|
||||
},
|
||||
"outputs": {
|
||||
"parameters": {
|
||||
"output_value": {
|
||||
"type": "STRING"
|
||||
}
|
||||
}
|
||||
},
|
||||
"executorLabel": "Producer"
|
||||
},
|
||||
{
|
||||
"executorLabel": "Consumer",
|
||||
"inputs": {
|
||||
"parameters": {
|
||||
"input_value": {
|
||||
"taskOutputParameter": {
|
||||
"producerTask": "Producer",
|
||||
"outputParameterKey": "output_value"
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"taskInfo": {
|
||||
"name": "Consumer"
|
||||
}
|
||||
}
|
||||
],
|
||||
"schemaVersion": "v2alpha1",
|
||||
"runtimeParameters": {
|
||||
"text": {
|
||||
"defaultValue": {
|
||||
"stringValue": "Hello world"
|
||||
},
|
||||
"type": "STRING"
|
||||
}
|
||||
},
|
||||
"pipelineInfo": {
|
||||
"name": "simple-two-step-pipeline-gongyuan-1609817103"
|
||||
},
|
||||
"deploymentConfig": {
|
||||
"executors": {
|
||||
"Consumer": {
|
||||
"container": {
|
||||
"image": "google/cloud-sdk:latest",
|
||||
"command": [
|
||||
"sh",
|
||||
"-c",
|
||||
"set -e -x\necho \"Read from an input parameter: \" && echo \"$0\"\n",
|
||||
"{{$.inputs.parameters['input_value']}}"
|
||||
]
|
||||
}
|
||||
},
|
||||
"Producer": {
|
||||
"container": {
|
||||
"image": "google/cloud-sdk:latest",
|
||||
"command": [
|
||||
"sh",
|
||||
"-c",
|
||||
"set -e -x\necho \"$0, this is an output parameter\" | gsutil cp - \"$1\"\n",
|
||||
"{{$.inputs.parameters['text']}}",
|
||||
"{{$.outputs.parameters['output_value'].output_file}}"
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"@type": "type.googleapis.com/ml_pipelines.PipelineDeploymentConfig"
|
||||
}
|
||||
},
|
||||
"runtimeConfig": {
|
||||
"gcsOutputDirectory": "gs://gongyuan-test/pipeline_root/gongyuan"
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,33 @@
|
|||
MLMD_VERSION=v0.26.0
|
||||
PROTO_DIR=ml_metadata/proto
|
||||
OUT_DIR=go_client
|
||||
|
||||
all: proto_gen
|
||||
|
||||
.PHONY: update
|
||||
update:
|
||||
# Please first edit MLMD_VERSION in Makefile
|
||||
$(MAKE) clean all
|
||||
|
||||
.PHONY: clean
|
||||
clean:
|
||||
rm -rf proto
|
||||
rm -rf $(OUT_DIR)
|
||||
|
||||
.PHONY: proto_gen
|
||||
proto_gen: $(PROTO_DIR)/metadata_store.proto $(PROTO_DIR)/metadata_store_service.proto
|
||||
\
|
||||
mkdir -p $(OUT_DIR) && \
|
||||
protoc -I=. --go_out=$(OUT_DIR) \
|
||||
--go-grpc_out=$(OUT_DIR) \
|
||||
$(PROTO_DIR)/metadata_store.proto $(PROTO_DIR)/metadata_store_service.proto \
|
||||
--go_opt=paths=source_relative \
|
||||
--go-grpc_opt=paths=source_relative
|
||||
|
||||
$(PROTO_DIR)/metadata_store.proto:
|
||||
mkdir -p ml_metadata/proto
|
||||
cd ml_metadata/proto && curl -LO https://raw.githubusercontent.com/google/ml-metadata/$(MLMD_VERSION)/ml_metadata/proto/metadata_store.proto
|
||||
|
||||
$(PROTO_DIR)/metadata_store_service.proto:
|
||||
mkdir -p ml_metadata/proto
|
||||
cd ml_metaddata/proto && curl -LO https://raw.githubusercontent.com/google/ml-metadata/$(MLMD_VERSION)/ml_metadata/proto/metadata_store_service.proto
|
||||
|
|
@ -0,0 +1,9 @@
|
|||
# Machine Learning Metadata
|
||||
|
||||
Upstream repo location: <https://github.com/google/ml-metadata>.
|
||||
|
||||
## Build golang gRPC client from proto
|
||||
|
||||
* [MLMD proto definitions](https://github.com/google/ml-metadata/tree/master/ml_metadata/proto)
|
||||
* [grpc go quickstart](https://grpc.io/docs/languages/go/quickstart/)
|
||||
* [client example code](https://github.com/grpc/grpc-go/blob/master/examples/helloworld/greeter_client/main.go)
|
||||
File diff suppressed because it is too large
Load Diff
6913
third_party/ml-metadata/go_client/ml_metadata/proto/metadata_store_service.pb.go
vendored
Normal file
6913
third_party/ml-metadata/go_client/ml_metadata/proto/metadata_store_service.pb.go
vendored
Normal file
File diff suppressed because it is too large
Load Diff
1875
third_party/ml-metadata/go_client/ml_metadata/proto/metadata_store_service_grpc.pb.go
vendored
Normal file
1875
third_party/ml-metadata/go_client/ml_metadata/proto/metadata_store_service_grpc.pb.go
vendored
Normal file
File diff suppressed because it is too large
Load Diff
|
|
@ -0,0 +1,689 @@
|
|||
/* Copyright 2019 Google LLC
|
||||
|
||||
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.
|
||||
==============================================================================*/
|
||||
syntax = "proto2";
|
||||
|
||||
package ml_metadata;
|
||||
|
||||
// A value in properties.
|
||||
message Value {
|
||||
// TODO(martinz): the types here may evolve over time.
|
||||
oneof value {
|
||||
int64 int_value = 1;
|
||||
double double_value = 2;
|
||||
string string_value = 3;
|
||||
}
|
||||
}
|
||||
|
||||
message Artifact {
|
||||
// Output only. The globally unique server generated id of the artifact.
|
||||
optional int64 id = 1;
|
||||
// The client provided name of the artifact. This field is optional. If set,
|
||||
// it must be unique among all the artifacts of the same artifact type within
|
||||
// a database instance and cannot be changed once set.
|
||||
optional string name = 7;
|
||||
// The id of an ArtifactType. This needs to be specified when an artifact is
|
||||
// created, and it cannot be changed.
|
||||
optional int64 type_id = 2;
|
||||
// Output only. The name of an ArtifactType.
|
||||
optional string type = 8;
|
||||
// The uniform resource identifier of the physical artifact.
|
||||
// May be empty if there is no physical artifact.
|
||||
optional string uri = 3;
|
||||
// Properties of the artifact.
|
||||
// Properties must be specified in the ArtifactType.
|
||||
map<string, Value> properties = 4;
|
||||
// User provided custom properties which are not defined by its type.
|
||||
map<string, Value> custom_properties = 5;
|
||||
|
||||
enum State {
|
||||
UNKNOWN = 0;
|
||||
// A state indicating that the artifact may exist.
|
||||
PENDING = 1;
|
||||
// A state indicating that the artifact should exist, unless something
|
||||
// external to the system deletes it.
|
||||
LIVE = 2;
|
||||
// A state indicating that the artifact should be deleted.
|
||||
MARKED_FOR_DELETION = 3;
|
||||
// A state indicating that the artifact has been deleted.
|
||||
DELETED = 4;
|
||||
}
|
||||
|
||||
// The state of the artifact known to the system.
|
||||
optional State state = 6;
|
||||
// Output only. Create time of the artifact in millisecond since epoch.
|
||||
optional int64 create_time_since_epoch = 9;
|
||||
// Output only. Last update time of the artifact since epoch in millisecond
|
||||
// since epoch.
|
||||
optional int64 last_update_time_since_epoch = 10;
|
||||
}
|
||||
|
||||
// TODO(martinz): consider moving this inside some message, to avoid having
|
||||
// literals directly in apo package.
|
||||
enum PropertyType {
|
||||
UNKNOWN = 0;
|
||||
INT = 1;
|
||||
DOUBLE = 2;
|
||||
STRING = 3;
|
||||
}
|
||||
|
||||
message ArtifactType {
|
||||
// The id of the type. 1-1 relationship between type names and IDs.
|
||||
optional int64 id = 1;
|
||||
// The name of the type. It must be unique among ArtifactTypes within a
|
||||
// database instance.
|
||||
optional string name = 2;
|
||||
// The schema of the type.
|
||||
// Properties are always optional in the artifact.
|
||||
// Properties of an artifact type can be expanded but not contracted (i.e.,
|
||||
// you can add columns but not remove them).
|
||||
map<string, PropertyType> properties = 3;
|
||||
}
|
||||
|
||||
// An event represents a relationship between an artifact and an execution.
|
||||
// There are different kinds of events, relating to both input and output, as
|
||||
// well as how they are used by the mlmd powered system.
|
||||
// For example, the DECLARED_INPUT and DECLARED_OUTPUT events are part of the
|
||||
// signature of an execution. For example, consider:
|
||||
//
|
||||
// my_result = my_execution({"data":[3,7],"schema":8})
|
||||
//
|
||||
// Where 3, 7, and 8 are artifact_ids, Assuming execution_id of my_execution is
|
||||
// 12 and artifact_id of my_result is 15, the events are:
|
||||
// {
|
||||
// artifact_id:3,
|
||||
// execution_id: 12,
|
||||
// type:DECLARED_INPUT,
|
||||
// path:{step:[{"key":"data"},{"index":0}]}
|
||||
// }
|
||||
// {
|
||||
// artifact_id:7,
|
||||
// execution_id: 12,
|
||||
// type:DECLARED_INPUT,
|
||||
// path:{step:[{"key":"data"},{"index":1}]}
|
||||
// }
|
||||
// {
|
||||
// artifact_id:8,
|
||||
// execution_id: 12,
|
||||
// type:DECLARED_INPUT,
|
||||
// path:{step:[{"key":"schema"}]}
|
||||
// }
|
||||
// {
|
||||
// artifact_id:15,
|
||||
// execution_id: 12,
|
||||
// type:DECLARED_OUTPUT,
|
||||
// path:{step:[{"key":"my_result"}]}
|
||||
// }
|
||||
// Other event types include INPUT/OUTPUT and INTERNAL_INPUT/_OUTPUT.
|
||||
// * The INPUT/OUTPUT is an event that actually reads/writes an artifact by an
|
||||
// execution. The input/output artifacts may not declared in the signature,
|
||||
// For example, the trainer may output multiple caches of the parameters
|
||||
// (as an OUTPUT), then finally write the SavedModel as a DECLARED_OUTPUT.
|
||||
// * The INTERNAL_INPUT/_OUTPUT are event types which are only meaningful to
|
||||
// an orchestration system to keep track of the details for later debugging.
|
||||
// For example, a fork happened conditioning on an artifact, then an execution
|
||||
// is triggered, such fork implementating may need to log the read and write
|
||||
// of artifacts and may not be worth displaying to the users.
|
||||
//
|
||||
// For instance, in the above example,
|
||||
//
|
||||
// my_result = my_execution({"data":[3,7],"schema":8})
|
||||
//
|
||||
// there is another execution (id: 15), which represents a `garbage_collection`
|
||||
// step in an orchestration system
|
||||
//
|
||||
// gc_result = garbage_collection(my_result)
|
||||
//
|
||||
// that cleans `my_result` if needed. The details should be invisible to the
|
||||
// end users and lineage tracking. The orchestrator can emit following events:
|
||||
//
|
||||
// {
|
||||
// artifact_id: 15,
|
||||
// execution_id: 15,
|
||||
// type:INTERNAL_INPUT,
|
||||
// }
|
||||
// {
|
||||
// artifact_id:16, // New artifact containing the GC job result.
|
||||
// execution_id: 15,
|
||||
// type:INTERNAL_OUTPUT,
|
||||
// path:{step:[{"key":"gc_result"}]}
|
||||
// }
|
||||
message Event {
|
||||
// A simple path (e.g. {step{key:"foo"}}) can name an artifact in the context
|
||||
// of an execution.
|
||||
message Path {
|
||||
message Step {
|
||||
oneof value {
|
||||
int64 index = 1;
|
||||
string key = 2;
|
||||
}
|
||||
}
|
||||
// A simple path (e.g. {step{key:"foo"}}) can name an artifact in the
|
||||
// context of an execution.
|
||||
repeated Step steps = 1;
|
||||
}
|
||||
// Events distinguish between an artifact that is written by the execution
|
||||
// (possibly as a cache), versus artifacts that are part of the declared
|
||||
// output of the Execution. For more information on what DECLARED_ means,
|
||||
// see the comment on the message.
|
||||
enum Type {
|
||||
UNKNOWN = 0;
|
||||
DECLARED_OUTPUT = 1; // A declared output of the execution.
|
||||
DECLARED_INPUT = 2; // A declared input of the execution.
|
||||
INPUT = 3; // An input of the execution.
|
||||
OUTPUT = 4; // An output of the execution.
|
||||
INTERNAL_INPUT = 5; // An internal input of the execution.
|
||||
INTERNAL_OUTPUT = 6; // An internal output of the execution.
|
||||
}
|
||||
// The artifact id is required for an event, and should refer to an
|
||||
// existing artifact.
|
||||
optional int64 artifact_id = 1;
|
||||
// The execution_id is required for an event, and should refer to an
|
||||
// existing execution.
|
||||
optional int64 execution_id = 2;
|
||||
// The path in an artifact struct, or the name of an artifact.
|
||||
optional Path path = 3;
|
||||
// The type of an event.
|
||||
optional Type type = 4;
|
||||
// Time the event occurred
|
||||
// Epoch is Jan 1, 1970, UTC
|
||||
optional int64 milliseconds_since_epoch = 5;
|
||||
}
|
||||
|
||||
message Execution {
|
||||
// Output only. The globally unique server generated id of the execution.
|
||||
optional int64 id = 1;
|
||||
// The client provided name of the execution. This field is optional. If set,
|
||||
// it must be unique among all the executions of the same artifact type within
|
||||
// a database instance and cannot be changed once set.
|
||||
optional string name = 6;
|
||||
// The id of an ExecutionType. This needs to be specified when an execution is
|
||||
// created, and it cannot be changed.
|
||||
// The id of an ExecutionType.
|
||||
optional int64 type_id = 2;
|
||||
// Output only. The name of an ExecutionType.
|
||||
optional string type = 7;
|
||||
// The state of the Execution. The state transitions are
|
||||
// NEW -> RUNNING -> COMPLETE | CACHED | FAILED | CANCELED
|
||||
// CACHED means the execution is skipped due to cached results.
|
||||
// CANCELED means the execution is skipped due to precondition not met. It is
|
||||
// different from CACHED in that a CANCELED execution will not have any event
|
||||
// associated with it. It is different from FAILED in that there is no
|
||||
// unexpected error happened and it is regarded as a normal state.
|
||||
enum State {
|
||||
UNKNOWN = 0;
|
||||
NEW = 1;
|
||||
RUNNING = 2;
|
||||
COMPLETE = 3;
|
||||
FAILED = 4;
|
||||
CACHED = 5;
|
||||
CANCELED = 6;
|
||||
}
|
||||
// The last known state of an execution in the system.
|
||||
optional State last_known_state = 3;
|
||||
// Properties of the Execution.
|
||||
// Properties must be specified in the ExecutionType.
|
||||
map<string, Value> properties = 4;
|
||||
// User provided custom properties which are not defined by its type.
|
||||
map<string, Value> custom_properties = 5;
|
||||
// Output only. Create time of the execution in millisecond since epoch.
|
||||
optional int64 create_time_since_epoch = 8;
|
||||
// Output only. Last update time of the execution in millisecond since epoch.
|
||||
optional int64 last_update_time_since_epoch = 9;
|
||||
}
|
||||
|
||||
message ExecutionType {
|
||||
// The id of the type. 1-1 relationship between type names and IDs.
|
||||
optional int64 id = 1;
|
||||
// The name of the type. It must be unique among ExecutionTypes within a
|
||||
// database instance.
|
||||
optional string name = 2;
|
||||
// The schema of the type.
|
||||
// Properties are always optional in the execution.
|
||||
map<string, PropertyType> properties = 3;
|
||||
// The ArtifactStructType of the input.
|
||||
// For example: {
|
||||
// "dict":{
|
||||
// "properties":{
|
||||
// "schema":{
|
||||
// "union_type":{
|
||||
// "none":{},
|
||||
// "simple":{...schema type...}
|
||||
// },
|
||||
// },
|
||||
// "data":{
|
||||
// "simple":{...data_type...}
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// That would be an optional schema field with a required data field.
|
||||
optional ArtifactStructType input_type = 4;
|
||||
// The ArtifactStructType of the output.
|
||||
// For example {"simple":{...stats gen output type...}}
|
||||
optional ArtifactStructType output_type = 5;
|
||||
}
|
||||
|
||||
message ContextType {
|
||||
// The id of the type. 1-1 relationship between type names and IDs.
|
||||
optional int64 id = 1;
|
||||
// The name of the type, e.g., Pipeline, Task, Session, User, etc. It must be
|
||||
// unique among ContextTypes within a database instance.
|
||||
optional string name = 2;
|
||||
// The schema of the type, e.g., name: string, owner: string
|
||||
// Properties are always optional in the context.
|
||||
// Properties of an context type can be expanded but not contracted (i.e.,
|
||||
// you can add columns but not remove them).
|
||||
map<string, PropertyType> properties = 3;
|
||||
}
|
||||
|
||||
message Context {
|
||||
// Output Only. The globally unique server generated id of the context.
|
||||
optional int64 id = 1;
|
||||
// The client provided name of the context. It must be unique within a
|
||||
// database instance.
|
||||
optional string name = 3;
|
||||
// The id of a ContextType. This needs to be specified when a context is
|
||||
// created, and it cannot be changed.
|
||||
optional int64 type_id = 2;
|
||||
// Output only. The name of a ContextType.
|
||||
optional string type = 6;
|
||||
// Values of the properties, which must be specified in the ContextType.
|
||||
map<string, Value> properties = 4;
|
||||
// User provided custom properties which are not defined by its type.
|
||||
map<string, Value> custom_properties = 5;
|
||||
// Output only. Create time of the context in millisecond since epoch.
|
||||
optional int64 create_time_since_epoch = 7;
|
||||
// Output only. Last update time of the context in millisecond since epoch.
|
||||
optional int64 last_update_time_since_epoch = 8;
|
||||
}
|
||||
|
||||
// the Attribution edges between Context and Artifact instances.
|
||||
message Attribution {
|
||||
optional int64 artifact_id = 1;
|
||||
optional int64 context_id = 2;
|
||||
}
|
||||
|
||||
// the Association edges between Context and Execution instances.
|
||||
message Association {
|
||||
optional int64 execution_id = 1;
|
||||
optional int64 context_id = 2;
|
||||
}
|
||||
|
||||
// the Parental Context edges between Context and Context instances.
|
||||
message ParentContext {
|
||||
optional int64 child_id = 1;
|
||||
optional int64 parent_id = 2;
|
||||
}
|
||||
|
||||
// The type of an ArtifactStruct.
|
||||
// An artifact struct type represents an infinite set of artifact structs.
|
||||
// It can specify the input or output type of an ExecutionType.
|
||||
// See the more specific types referenced in the message for more details.
|
||||
message ArtifactStructType {
|
||||
oneof kind {
|
||||
ArtifactType simple = 1; // Matches exactly this type.
|
||||
UnionArtifactStructType union_type = 2;
|
||||
IntersectionArtifactStructType intersection = 3;
|
||||
ListArtifactStructType list = 4;
|
||||
NoneArtifactStructType none = 5;
|
||||
AnyArtifactStructType any = 6;
|
||||
TupleArtifactStructType tuple = 7;
|
||||
DictArtifactStructType dict = 8;
|
||||
}
|
||||
}
|
||||
|
||||
// Represents a union of types.
|
||||
message UnionArtifactStructType {
|
||||
// An artifact struct matches this type if it matches any of the candidates.
|
||||
// If candidates is empty, this is a bottom type (matches no artifacts).
|
||||
repeated ArtifactStructType candidates = 1;
|
||||
}
|
||||
|
||||
// A member of this type must satisfy all constraints.
|
||||
// This primarily useful not as an end-user type, but something calculated
|
||||
// as an intermediate type in the system.
|
||||
//
|
||||
// For example, suppose you have a method:
|
||||
// def infer_my_input_type(a): # try to infer the input type of this method.
|
||||
// use_in_method_x(a) # with input type x_input
|
||||
// use_in_method_y(a) # with input type y_input
|
||||
//
|
||||
// Given this information, you know that infer_my_input_type has
|
||||
// type {"intersection":{"constraints":[x_input, y_input]}}.
|
||||
//
|
||||
// IntersectionArtifactStructType intersection_type = {"constraints":[
|
||||
// {"dict":{"properties":{"schema":{"any":{}}},
|
||||
// "extra_properties":{"any":{}}}},
|
||||
// {"dict":{"properties":{"data":{"any":{}}},
|
||||
// "extra_properties":{"any":{}}}}]}
|
||||
// Since the first constraint requires the dictionary to have a schema
|
||||
// property, and the second constraint requires it to have a data property, this
|
||||
// is equivalent to:
|
||||
// ArtifactStructType other_type =
|
||||
// {"dict":{"properties":{"schema":{"any":{}},"data":{"any":{}}}},
|
||||
// "extra_properties":{"any":{}}}
|
||||
//
|
||||
message IntersectionArtifactStructType {
|
||||
repeated ArtifactStructType constraints = 1;
|
||||
}
|
||||
|
||||
// Represents an ArtifactStruct list type with homogeneous elements.
|
||||
message ListArtifactStructType {
|
||||
// Every entry in the list must be of this type.
|
||||
// Note: if this type is Any, then the list can have arbitrary elements.
|
||||
optional ArtifactStructType element = 1;
|
||||
}
|
||||
|
||||
// The only member of this type is a None artifact.
|
||||
// Note: ArtifactStruct{} is a None artifact.
|
||||
// This can represent an execution that has no outputs (or inputs),
|
||||
// or can be part of a UnionArtifactStructType to represent an optional
|
||||
// input.
|
||||
// For example, StatsGen has an "optional" schema input.
|
||||
// A practical example of this is:
|
||||
// stats_gen_type = {
|
||||
// "dict":{
|
||||
// "properties":{
|
||||
// "schema":{
|
||||
// "union_type":{
|
||||
// "none":{},
|
||||
// "simple":{...schema type...}
|
||||
// },
|
||||
// },
|
||||
// "data":{
|
||||
// "simple":{...data_type...}
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// };
|
||||
message NoneArtifactStructType {}
|
||||
|
||||
// Every ArtifactStruct is a member of this type.
|
||||
message AnyArtifactStructType {}
|
||||
|
||||
// An ordered list of heterogeneous artifact structs.
|
||||
// The length of the list is fixed.
|
||||
// Each position in the list can have a different type.
|
||||
message TupleArtifactStructType {
|
||||
repeated ArtifactStructType elements = 1;
|
||||
}
|
||||
|
||||
// A artifact struct type that represents a record or struct-like dictionary.
|
||||
// ArtifactStruct would be map (i.e. ArtifactStructMap)
|
||||
message DictArtifactStructType {
|
||||
// Underlying properties for the type.
|
||||
map<string, ArtifactStructType> properties = 1;
|
||||
|
||||
// If true, then if properties["foo"] can be None, then that key is not
|
||||
// required.
|
||||
optional bool none_type_not_required = 2;
|
||||
|
||||
// Extra keys are allowed that are not specified in properties. These
|
||||
// keys must have the type specified below.
|
||||
// If this is not specified, then extra properties are not allowed.
|
||||
optional ArtifactStructType extra_properties_type = 3;
|
||||
}
|
||||
|
||||
// Configuration for a "fake" database.
|
||||
// This database is an in-memory SQLite database that lives only as
|
||||
// long as the associated object lives.
|
||||
message FakeDatabaseConfig {}
|
||||
|
||||
message MySQLDatabaseConfig {
|
||||
// The hostname or IP address of the MYSQL server:
|
||||
// * If unspecified, a connection to the local host is assumed.
|
||||
// The client connects using a Unix socket specified by `socket`.
|
||||
// * Otherwise, TCP/IP is used.
|
||||
// Currently a replicated MYSQL backend is not supported.
|
||||
optional string host = 1;
|
||||
// The TCP Port number that the MYSQL server accepts connections on.
|
||||
// If unspecified, the default MYSQL port (3306) is used.
|
||||
optional uint32 port = 2;
|
||||
// The database to connect to. Must be specified.
|
||||
// After connecting to the MYSQL server, this database is created if not
|
||||
// already present.
|
||||
// All queries after Connect() are assumed to be for this database.
|
||||
optional string database = 3;
|
||||
// The MYSQL login id. If empty, the current user is assumed.
|
||||
optional string user = 4;
|
||||
// The password to use for `user`. If empty, only MYSQL user ids that don't
|
||||
// have a password set are allowed to connect.
|
||||
optional string password = 5;
|
||||
// The Unix socket to use to connect to the server. If unspecified, a
|
||||
// `host` must be provided.
|
||||
optional string socket = 6;
|
||||
|
||||
// The options to establish encrypted connections to MySQL using SSL.
|
||||
message SSLOptions {
|
||||
// The path name of the client private key file.
|
||||
optional string key = 1;
|
||||
// The path name of the client public key certificate file.
|
||||
optional string cert = 2;
|
||||
// The path name of the CA certificate file.
|
||||
optional string ca = 3;
|
||||
// The path name of the directory that contains trusted SSL CA certificates.
|
||||
optional string capath = 4;
|
||||
// The list of permissible ciphers for SSL encryption.
|
||||
optional string cipher = 5;
|
||||
// If set, enable verification of the server certificate against the host
|
||||
// name used when connecting to the server.
|
||||
optional bool verify_server_cert = 6;
|
||||
}
|
||||
// If the field is set, the ssl options are set in mysql_options before
|
||||
// establishing a connection. It is ignored if the mysql server does not
|
||||
// enable SSL.
|
||||
optional SSLOptions ssl_options = 7;
|
||||
}
|
||||
|
||||
// A config contains the parameters when using with SqliteMetadatSource.
|
||||
message SqliteMetadataSourceConfig {
|
||||
// A uri specifying Sqlite3 database filename, for example:
|
||||
//
|
||||
// file:some_sqlite3_file_in_local_dir.db
|
||||
// file:///home/username/some_sqlite3_file.db
|
||||
//
|
||||
// see https://www.sqlite.org/c3ref/open.html for model details
|
||||
//
|
||||
// If not given, a in-memory sqlite3 database is used, and destroyed when
|
||||
// disconnecting the metadata source.
|
||||
optional string filename_uri = 1;
|
||||
|
||||
// Connection parameters for SQLite3 based metadata source.
|
||||
enum ConnectionMode {
|
||||
UNKNOWN = 0;
|
||||
// Connect a metadata source in read-only mode. Connection fail if the
|
||||
// sqlite3 database at the `filename` does not exist. Any queries modifying
|
||||
// the database fail.
|
||||
READONLY = 1;
|
||||
// Connect a metadata source in read/write mode. Connection fail if the
|
||||
// sqlite3 database at the `filename` does not exist.
|
||||
READWRITE = 2;
|
||||
// Similar to READWRITE. In addition, it creates the database if it does not
|
||||
// exist.
|
||||
READWRITE_OPENCREATE = 3;
|
||||
}
|
||||
|
||||
// A flag specifying the connection mode. If not given, default connection
|
||||
// mode is set to READWRITE_OPENCREATE.
|
||||
optional ConnectionMode connection_mode = 2;
|
||||
}
|
||||
|
||||
|
||||
message MigrationOptions {
|
||||
// If not set, by default the upgrade migration is disabled. MLMD only
|
||||
// compares db_v with the lib_v, and raise error if the two do not align.
|
||||
// If the field is set to true, MLMD performs upgrade migration. It upgrades
|
||||
// the database schema version (db_v) to align with the library schema
|
||||
// version (lib_v) when connecting to the database.
|
||||
// Schema migration should not be run concurrently with multiple clients to
|
||||
// prevent data races.
|
||||
optional bool enable_upgrade_migration = 3;
|
||||
|
||||
// Downgrade the given database to the specified schema version.
|
||||
// For v0.13.2 release, the schema_version is 0.
|
||||
// For 0.14.0 and 0.15.0 release, the schema_version is 4.
|
||||
// More details are described in g3doc/get_start.md#upgrade-mlmd-library
|
||||
// Set this field only when a database is accidentally upgraded by a newer
|
||||
// version library. Each library version only knows how to downgrade to
|
||||
// previous schema versions. As downgrade migrations inevitably introduce
|
||||
// data loss, please consider taking a backup of the database before
|
||||
// downgrading schema.
|
||||
// After downgrade migration, the database connection is canceled. The user
|
||||
// needs to downgrade the library to use the database.
|
||||
optional int64 downgrade_to_schema_version = 2 [default = -1];
|
||||
|
||||
reserved 1;
|
||||
}
|
||||
|
||||
message RetryOptions {
|
||||
// The max number of retries when transaction returns Aborted error.
|
||||
optional int64 max_num_retries = 1;
|
||||
}
|
||||
|
||||
message ConnectionConfig {
|
||||
// Configuration for a new connection.
|
||||
oneof config {
|
||||
FakeDatabaseConfig fake_database = 1;
|
||||
MySQLDatabaseConfig mysql = 2;
|
||||
SqliteMetadataSourceConfig sqlite = 3;
|
||||
}
|
||||
|
||||
// Options for overwriting the default retry setting when MLMD transactions
|
||||
// returning Aborted error.
|
||||
// The setting is currently available for python client library only.
|
||||
// TODO(b/154862807) set the setting in transaction executor.
|
||||
optional RetryOptions retry_options = 4;
|
||||
}
|
||||
|
||||
// A list of supported GRPC arguments defined in:
|
||||
// https://grpc.github.io/grpc/core/group__grpc__arg__keys.html
|
||||
message GrpcChannelArguments {
|
||||
// Maximum message length in bytes per response that the channel can receive.
|
||||
optional int64 max_receive_message_length = 1;
|
||||
}
|
||||
|
||||
// Configuration for the gRPC metadata store client.
|
||||
message MetadataStoreClientConfig {
|
||||
// The hostname or IP address of the gRPC server. Must be specified.
|
||||
optional string host = 1;
|
||||
// The TCP Port number that the gRPC server accepts connections on.
|
||||
// Must be specified.
|
||||
optional uint32 port = 2;
|
||||
|
||||
message SSLConfig {
|
||||
// The PEM-encoded private key as a byte string, or Empty if no private key
|
||||
// should be used.
|
||||
optional string client_key = 1;
|
||||
// The PEM-encoded certificate chain as a byte string to use or or Empty if
|
||||
// no certificate chain should be used.
|
||||
optional string server_cert = 2;
|
||||
// The PEM-encoded root certificates as a byte string, or Empty to retrieve
|
||||
// them from a default location chosen by gRPC runtime.
|
||||
optional string custom_ca = 3;
|
||||
}
|
||||
|
||||
// Configuration for a secure gRPC channel.
|
||||
// If not given, insecure connection is used.
|
||||
optional SSLConfig ssl_config = 3;
|
||||
|
||||
// GRPC channel creation arguments.
|
||||
optional GrpcChannelArguments channel_arguments = 4;
|
||||
|
||||
// Time duration that a client is willing to wait for a reply from the server.
|
||||
// If unset, the timeout is considered infinite. When the field is specified,
|
||||
// Grpc APIs would return DeadlineExceededError when server does not respond
|
||||
// within `client_timeout_sec`. Floating point valued, in seconds.
|
||||
optional double client_timeout_sec = 5;
|
||||
|
||||
}
|
||||
|
||||
// Configuration for the gRPC metadata store server.
|
||||
message MetadataStoreServerConfig {
|
||||
// Configuration to connect the metadata source backend.
|
||||
optional ConnectionConfig connection_config = 1;
|
||||
|
||||
// Configuration for upgrade and downgrade migrations the metadata source.
|
||||
optional MigrationOptions migration_options = 3;
|
||||
|
||||
message SSLConfig {
|
||||
// Private server key for SSL
|
||||
optional string server_key = 1;
|
||||
// Public server certificate
|
||||
optional string server_cert = 2;
|
||||
// Custom certificate authority
|
||||
optional string custom_ca = 3;
|
||||
// Valid client certificate required?
|
||||
optional bool client_verify = 4;
|
||||
}
|
||||
|
||||
// Configuration for a secure gRPC channel.
|
||||
// If not given, insecure connection is used.
|
||||
optional SSLConfig ssl_config = 2;
|
||||
}
|
||||
|
||||
// ListOperationOptions represents the set of options and predicates to be
|
||||
// used for List operations on Artifacts, Executions and Contexts.
|
||||
message ListOperationOptions {
|
||||
// Max number of resources to return in the result. A value of zero or less
|
||||
// results in a InvalidArgumentError.
|
||||
// The API implementation also enforces an upper-bound of 100, and picks the
|
||||
// minimum between this value and the one specified here.
|
||||
optional int32 max_result_size = 1 [default = 20];
|
||||
|
||||
message OrderByField {
|
||||
// Supported fields for Ordering.
|
||||
enum Field {
|
||||
FIELD_UNSPECIFIED = 0;
|
||||
CREATE_TIME = 1;
|
||||
LAST_UPDATE_TIME = 2;
|
||||
ID = 3;
|
||||
}
|
||||
|
||||
// Field to order.
|
||||
optional Field field = 1 [default = ID];
|
||||
|
||||
// Direction of ordering.
|
||||
optional bool is_asc = 2 [default = true];
|
||||
}
|
||||
|
||||
// Ordering field.
|
||||
optional OrderByField order_by_field = 2;
|
||||
|
||||
// Identifies the next page of results.
|
||||
optional string next_page_token = 3;
|
||||
|
||||
}
|
||||
|
||||
// Encapsulates information to identify the next page of resources in
|
||||
// ListOperation.
|
||||
message ListOperationNextPageToken {
|
||||
// Id offset within the resultset to start next page.
|
||||
// Id offset is returned as Id is the unique field used to break ties for
|
||||
// fields that might have duplicate entries, e.g. there could be two
|
||||
// resources with same last_update_time. In such cases to break the tie in
|
||||
// ordering, id offset is used. Also if during the calls, the node is updated,
|
||||
// and its last_update_time has been changed, running the pagination query
|
||||
// again, the updated node will moved to earlier pages.
|
||||
optional int64 id_offset = 1;
|
||||
|
||||
// Offset value of the order by field. If ID is used this value is same as
|
||||
// id_offset.
|
||||
optional int64 field_offset = 2;
|
||||
|
||||
// Options set in the first call to ListOperation. This ensures that if
|
||||
// next_page_token is set by the caller then ListPipelineJobs API will always
|
||||
// use options set in the first call.
|
||||
optional ListOperationOptions set_options = 3;
|
||||
}
|
||||
|
|
@ -0,0 +1,875 @@
|
|||
/* Copyright 2019 Google LLC
|
||||
|
||||
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.
|
||||
==============================================================================*/
|
||||
|
||||
syntax = "proto2";
|
||||
|
||||
package ml_metadata;
|
||||
|
||||
import "ml_metadata/proto/metadata_store.proto";
|
||||
|
||||
// An artifact and type pair. Part of an artifact struct.
|
||||
message ArtifactAndType {
|
||||
optional Artifact artifact = 1;
|
||||
optional ArtifactType type = 2;
|
||||
}
|
||||
|
||||
// A dictionary of artifact structs. Can represent a dictionary.
|
||||
message ArtifactStructMap {
|
||||
// An artifact struct that is a dictionary.
|
||||
// Can be represented as a JSON dictionary of artifact structs.
|
||||
map<string, ArtifactStruct> properties = 1;
|
||||
}
|
||||
|
||||
// An artifact struct that is a list.
|
||||
message ArtifactStructList {
|
||||
// Can be represented as a JSON list of artifact structs.
|
||||
repeated ArtifactStruct elements = 1;
|
||||
}
|
||||
|
||||
// An artifact struct represents the input or output of an Execution.
|
||||
// See the more specific types referenced in the message for more details.
|
||||
message ArtifactStruct {
|
||||
// Note: an artifact struct may be empty to indicate "None" or null.
|
||||
oneof value {
|
||||
ArtifactAndType artifact = 1;
|
||||
ArtifactStructMap map = 2;
|
||||
ArtifactStructList list = 3;
|
||||
}
|
||||
}
|
||||
|
||||
message PutArtifactsRequest {
|
||||
repeated Artifact artifacts = 1;
|
||||
|
||||
message Options {
|
||||
// When there are multiple writers to update an existing node to
|
||||
// different states, there may be a race and the end result of the
|
||||
// concurrent update is nondeterministic. If the field is set, then an
|
||||
// optimistic concurrency control (OCC) scheme is used during update:
|
||||
// it compares the `artifact`.`last_update_time_since_epoch` in the request
|
||||
// with the stored `last_update_time_since_epoch` having the same
|
||||
// `artifact`.`id`. If they are different, the request fails, and the user
|
||||
// can read the stored node and retry node update.
|
||||
// When the option is set, the timestamp after update is guaranteed to be
|
||||
// increased and different from the input artifact.
|
||||
// When set the option, the caller should set it for all concurrent writers.
|
||||
optional bool abort_if_latest_updated_time_changed = 1;
|
||||
}
|
||||
|
||||
// Additional options to change the behavior of the method.
|
||||
optional Options options = 2;
|
||||
}
|
||||
|
||||
message PutArtifactsResponse {
|
||||
// A list of artifact ids index-aligned with PutArtifactsRequest.
|
||||
repeated int64 artifact_ids = 1;
|
||||
}
|
||||
|
||||
message PutArtifactTypeRequest {
|
||||
// The field is required in any request. Stored types in MLMD can be updated
|
||||
// by introducing new properties and remain backward compatible. If a type
|
||||
// with the same name exists in the database, it updates the existing type,
|
||||
// otherwise it creates a new type.
|
||||
optional ArtifactType artifact_type = 1;
|
||||
|
||||
// If true then allows adding properties to an existing stored type.
|
||||
// If false, then type update is not allowed and it raises AlreadyExists
|
||||
// error if the given type has any new property that is not defined in the
|
||||
// stored type.
|
||||
optional bool can_add_fields = 2;
|
||||
|
||||
// If true then allows omitting properties of an existing stored type.
|
||||
// If false, then no properties of the stored type can be omitted in the
|
||||
// given type, otherwise it raises AlreadyExists error.
|
||||
optional bool can_omit_fields = 5;
|
||||
|
||||
// Deprecated fields.
|
||||
optional bool can_delete_fields = 3 [deprecated = true];
|
||||
optional bool all_fields_match = 4 [default = true, deprecated = true];
|
||||
}
|
||||
|
||||
message PutArtifactTypeResponse {
|
||||
// The type ID of the artifact type.
|
||||
optional int64 type_id = 1;
|
||||
}
|
||||
|
||||
message PutExecutionsRequest {
|
||||
repeated Execution executions = 1;
|
||||
}
|
||||
|
||||
message PutExecutionsResponse {
|
||||
// A list of execution ids index-aligned with PutExecutionsRequest.
|
||||
repeated int64 execution_ids = 1;
|
||||
}
|
||||
|
||||
message PutExecutionTypeRequest {
|
||||
// The field is required in any request. Stored types in MLMD can be updated
|
||||
// by introducing new properties and remain backward compatible. If a type
|
||||
// with the same name exists in the database, it updates the existing type,
|
||||
// otherwise it creates a new type.
|
||||
optional ExecutionType execution_type = 1;
|
||||
|
||||
// If true then allows adding properties to an existing stored type.
|
||||
// If false, then type update is not allowed and it raises AlreadyExists
|
||||
// error if the given type has any new property that is not defined in the
|
||||
// stored type.
|
||||
optional bool can_add_fields = 2;
|
||||
|
||||
// If true then allows omitting properties of an existing stored type.
|
||||
// If false, then no properties of the stored type can be omitted in the
|
||||
// given type, otherwise it raises AlreadyExists error.
|
||||
optional bool can_omit_fields = 5;
|
||||
|
||||
// Deprecated fields.
|
||||
optional bool can_delete_fields = 3 [deprecated = true];
|
||||
optional bool all_fields_match = 4 [default = true, deprecated = true];
|
||||
}
|
||||
|
||||
message PutExecutionTypeResponse {
|
||||
// The type ID of the execution type.
|
||||
optional int64 type_id = 1;
|
||||
}
|
||||
|
||||
message PutEventsRequest {
|
||||
repeated Event events = 1;
|
||||
}
|
||||
|
||||
message PutEventsResponse {}
|
||||
|
||||
message PutExecutionRequest {
|
||||
// A pair of an artifact and an event used or generated by an execution, e.g.,
|
||||
// during the execution run, it uses none or many artifacts as input, and
|
||||
// generate none or many artifacts as output.
|
||||
message ArtifactAndEvent {
|
||||
// The pair may have an artifact. If present and no artifact.id is given,
|
||||
// then it inserts the artifact, otherwise it updates the artifact.
|
||||
optional Artifact artifact = 1;
|
||||
// The pair may have an event. Providing event.artifact_id or
|
||||
// event.execution_id is optional. If the ids are given, it must align with
|
||||
// the `artifact`.id / `execution`.id respectively. If artifact is not
|
||||
// given and event.artifact_id is set, it must exist in the backend.
|
||||
optional Event event = 2;
|
||||
}
|
||||
|
||||
message Options {
|
||||
// When there's a race to publish executions with a new context with the
|
||||
// same context.name, by default there'll be one writer succeeds and
|
||||
// the rest of the writers returning AlreadyExists errors. If set the field,
|
||||
// the failed writer will reuse the stored context in the transaction.
|
||||
optional bool reuse_context_if_already_exist = 1;
|
||||
}
|
||||
// The execution that produces many artifact and event pairs.
|
||||
optional Execution execution = 1;
|
||||
// The list of artifact and event pairs.
|
||||
repeated ArtifactAndEvent artifact_event_pairs = 2;
|
||||
// A list of contexts associated with the execution and artifacts. For each
|
||||
// given context without a context.id, it inserts the context, otherwise it
|
||||
// updates the stored context with the same id.
|
||||
// Associations between each pair of contexts and the execution, and
|
||||
// attributions between each pair of contexts and artifacts are created if
|
||||
// they do not already exist.
|
||||
repeated Context contexts = 3;
|
||||
// Additional options to change the behavior of the method.
|
||||
optional Options options = 4;
|
||||
}
|
||||
|
||||
message PutExecutionResponse {
|
||||
// An execution id of the `execution` in PutExecutionRequest.
|
||||
optional int64 execution_id = 1;
|
||||
// A list of artifact ids index-aligned with `artifact_event_pairs` in the
|
||||
// PutExecutionRequest.
|
||||
repeated int64 artifact_ids = 2;
|
||||
// A list of context ids index-aligned with `contexts` in the
|
||||
// PutExecutionRequest.
|
||||
repeated int64 context_ids = 3;
|
||||
}
|
||||
|
||||
message PutTypesRequest {
|
||||
repeated ArtifactType artifact_types = 1;
|
||||
repeated ExecutionType execution_types = 2;
|
||||
repeated ContextType context_types = 3;
|
||||
|
||||
// If true then allows adding properties to an existing stored type.
|
||||
// If false, then type update is not allowed and it raises AlreadyExists
|
||||
// error if the given type has any new property that is not defined in the
|
||||
// stored type.
|
||||
optional bool can_add_fields = 4;
|
||||
|
||||
// If true then allows omitting properties of an existing stored type.
|
||||
// If false, then no properties of the stored type can be omitted in the
|
||||
// given type, otherwise it raises AlreadyExists error.
|
||||
optional bool can_omit_fields = 7;
|
||||
|
||||
// Deprecated fields.
|
||||
optional bool can_delete_fields = 5 [deprecated = true];
|
||||
optional bool all_fields_match = 6 [default = true, deprecated = true];
|
||||
}
|
||||
|
||||
message PutTypesResponse {
|
||||
// The type ids of the artifact type.
|
||||
repeated int64 artifact_type_ids = 1;
|
||||
// The type ids of the execution type.
|
||||
repeated int64 execution_type_ids = 2;
|
||||
// The type ids of the context type.
|
||||
repeated int64 context_type_ids = 3;
|
||||
}
|
||||
|
||||
message PutContextTypeRequest {
|
||||
// The field is required in any request. Stored types in MLMD can be updated
|
||||
// by introducing new properties and remain backward compatible. If a type
|
||||
// with the same name exists in the database, it updates the existing type,
|
||||
// otherwise it creates a new type.
|
||||
optional ContextType context_type = 1;
|
||||
|
||||
// If true then allows adding properties to an existing stored type.
|
||||
// If false, then type update is not allowed and it raises AlreadyExists
|
||||
// error if the given type has any new property that is not defined in the
|
||||
// stored type.
|
||||
optional bool can_add_fields = 2;
|
||||
|
||||
// If true then allows omitting properties of an existing stored type.
|
||||
// If false, then no properties of the stored type can be omitted in the
|
||||
// given type, otherwise it raises AlreadyExists error.
|
||||
optional bool can_omit_fields = 5;
|
||||
|
||||
// Deprecated fields.
|
||||
optional bool can_delete_fields = 3 [deprecated = true];
|
||||
optional bool all_fields_match = 4 [default = true, deprecated = true];
|
||||
}
|
||||
|
||||
message PutContextTypeResponse {
|
||||
// The type ID of the context type.
|
||||
optional int64 type_id = 1;
|
||||
}
|
||||
|
||||
message PutContextsRequest {
|
||||
repeated Context contexts = 1;
|
||||
}
|
||||
|
||||
message PutContextsResponse {
|
||||
// A list of context ids index-aligned with PutContextsRequest.
|
||||
repeated int64 context_ids = 1;
|
||||
}
|
||||
|
||||
message PutAttributionsAndAssociationsRequest {
|
||||
repeated Attribution attributions = 1;
|
||||
repeated Association associations = 2;
|
||||
}
|
||||
|
||||
message PutAttributionsAndAssociationsResponse {}
|
||||
|
||||
message PutParentContextsRequest {
|
||||
repeated ParentContext parent_contexts = 1;
|
||||
}
|
||||
|
||||
message PutParentContextsResponse {}
|
||||
|
||||
message GetArtifactsByTypeRequest {
|
||||
optional string type_name = 1;
|
||||
}
|
||||
|
||||
message GetArtifactsByTypeResponse {
|
||||
repeated Artifact artifacts = 1;
|
||||
}
|
||||
|
||||
message GetArtifactByTypeAndNameRequest {
|
||||
optional string type_name = 1;
|
||||
optional string artifact_name = 2;
|
||||
}
|
||||
|
||||
message GetArtifactByTypeAndNameResponse {
|
||||
optional Artifact artifact = 1;
|
||||
}
|
||||
|
||||
message GetArtifactsByIDRequest {
|
||||
// A list of artifact ids to retrieve.
|
||||
repeated int64 artifact_ids = 1;
|
||||
}
|
||||
|
||||
message GetArtifactsByIDResponse {
|
||||
// Artifacts with matching ids.
|
||||
// This is not index-aligned: if an id is not found, it is not
|
||||
repeated Artifact artifacts = 1;
|
||||
}
|
||||
|
||||
// Request to retrieve Artifacts using List options.
|
||||
// If option is not specified then all Artifacts are returned.
|
||||
message GetArtifactsRequest {
|
||||
// Specify options.
|
||||
// Currently supports:
|
||||
// 1. Field to order the results.
|
||||
// 2. Page size.
|
||||
optional ListOperationOptions options = 1;
|
||||
}
|
||||
|
||||
message GetArtifactsResponse {
|
||||
// Returned artifacts.
|
||||
repeated Artifact artifacts = 1;
|
||||
|
||||
// Token to use to retreive next page of results if list options are used in
|
||||
// the request.
|
||||
optional string next_page_token = 2;
|
||||
}
|
||||
|
||||
message GetArtifactsByURIRequest {
|
||||
// A list of artifact uris to retrieve.
|
||||
repeated string uris = 2;
|
||||
|
||||
reserved 1;
|
||||
}
|
||||
|
||||
message GetArtifactsByURIResponse {
|
||||
repeated Artifact artifacts = 1;
|
||||
}
|
||||
|
||||
// Request to retrieve Executions using List options.
|
||||
// If option is not specified then all Executions are returned.
|
||||
message GetExecutionsRequest {
|
||||
// Specify options.
|
||||
// Currently supports:
|
||||
// 1. Field to order the results.
|
||||
// 2. Page size.
|
||||
optional ListOperationOptions options = 1;
|
||||
}
|
||||
|
||||
message GetExecutionsResponse {
|
||||
// Returned executions.
|
||||
repeated Execution executions = 1;
|
||||
|
||||
// Token to use to retreive next page of results if list options are used in
|
||||
// the request.
|
||||
optional string next_page_token = 2;
|
||||
}
|
||||
|
||||
message GetArtifactTypeRequest {
|
||||
optional string type_name = 1;
|
||||
}
|
||||
|
||||
message GetArtifactTypeResponse {
|
||||
// Gets an artifact type, or clear if it does not exist.
|
||||
optional ArtifactType artifact_type = 1;
|
||||
}
|
||||
|
||||
message GetArtifactTypesRequest {}
|
||||
|
||||
message GetArtifactTypesResponse {
|
||||
repeated ArtifactType artifact_types = 1;
|
||||
}
|
||||
|
||||
message GetExecutionTypesRequest {}
|
||||
|
||||
message GetExecutionTypesResponse {
|
||||
repeated ExecutionType execution_types = 1;
|
||||
}
|
||||
|
||||
message GetContextTypesRequest {}
|
||||
|
||||
message GetContextTypesResponse {
|
||||
repeated ContextType context_types = 1;
|
||||
}
|
||||
|
||||
message GetExecutionsByTypeRequest {
|
||||
optional string type_name = 1;
|
||||
}
|
||||
|
||||
message GetExecutionsByTypeResponse {
|
||||
repeated Execution executions = 1;
|
||||
}
|
||||
|
||||
message GetExecutionByTypeAndNameRequest {
|
||||
optional string type_name = 1;
|
||||
optional string execution_name = 2;
|
||||
}
|
||||
|
||||
message GetExecutionByTypeAndNameResponse {
|
||||
optional Execution execution = 1;
|
||||
}
|
||||
|
||||
message GetExecutionsByIDRequest {
|
||||
// A list of execution ids to retrieve.
|
||||
repeated int64 execution_ids = 1;
|
||||
}
|
||||
|
||||
message GetExecutionsByIDResponse {
|
||||
// The result is not index-aligned: if an id is not found, it is not
|
||||
// returned.
|
||||
repeated Execution executions = 1;
|
||||
}
|
||||
|
||||
message GetExecutionTypeRequest {
|
||||
optional string type_name = 1;
|
||||
}
|
||||
|
||||
message GetExecutionTypeResponse {
|
||||
// Gets an execution type, or clear if it does not exist.
|
||||
optional ExecutionType execution_type = 1;
|
||||
}
|
||||
|
||||
// Gets all events with matching execution ids.
|
||||
message GetEventsByExecutionIDsRequest {
|
||||
repeated int64 execution_ids = 1;
|
||||
}
|
||||
|
||||
message GetEventsByExecutionIDsResponse {
|
||||
repeated Event events = 1;
|
||||
}
|
||||
|
||||
message GetEventsByArtifactIDsRequest {
|
||||
repeated int64 artifact_ids = 1;
|
||||
}
|
||||
|
||||
message GetEventsByArtifactIDsResponse {
|
||||
repeated Event events = 1;
|
||||
}
|
||||
|
||||
message GetArtifactTypesByIDRequest {
|
||||
repeated int64 type_ids = 1;
|
||||
}
|
||||
|
||||
message GetArtifactTypesByIDResponse {
|
||||
// The result is not index-aligned: if an id is not found, it is not
|
||||
// returned.
|
||||
repeated ArtifactType artifact_types = 1;
|
||||
}
|
||||
|
||||
message GetExecutionTypesByIDRequest {
|
||||
repeated int64 type_ids = 1;
|
||||
}
|
||||
|
||||
message GetExecutionTypesByIDResponse {
|
||||
// The result is not index-aligned: if an id is not found, it is not
|
||||
// returned.
|
||||
repeated ExecutionType execution_types = 1;
|
||||
}
|
||||
|
||||
message GetContextTypeRequest {
|
||||
optional string type_name = 1;
|
||||
}
|
||||
|
||||
message GetContextTypeResponse {
|
||||
// Gets a context type, or clear if it does not exist.
|
||||
optional ContextType context_type = 1;
|
||||
}
|
||||
|
||||
message GetContextTypesByIDRequest {
|
||||
repeated int64 type_ids = 1;
|
||||
}
|
||||
|
||||
message GetContextTypesByIDResponse {
|
||||
// The result is not index-aligned: if an id is not found, it is not
|
||||
// returned.
|
||||
repeated ContextType context_types = 1;
|
||||
}
|
||||
|
||||
// Request to retrieve Contexts using List options.
|
||||
// If option is not specified then all Contexts are returned.
|
||||
message GetContextsRequest {
|
||||
// Specify options.
|
||||
// Currently supports:
|
||||
// 1. Field to order the results.
|
||||
// 2. Page size.
|
||||
optional ListOperationOptions options = 1;
|
||||
}
|
||||
|
||||
message GetContextsResponse {
|
||||
// Returned contexts.
|
||||
repeated Context contexts = 1;
|
||||
|
||||
// Token to use to retreive next page of results if list options are used in
|
||||
// the request.
|
||||
optional string next_page_token = 2;
|
||||
}
|
||||
|
||||
message GetContextsByTypeRequest {
|
||||
optional string type_name = 1;
|
||||
}
|
||||
|
||||
message GetContextsByTypeResponse {
|
||||
repeated Context contexts = 1;
|
||||
}
|
||||
|
||||
message GetContextByTypeAndNameRequest {
|
||||
optional string type_name = 1;
|
||||
optional string context_name = 2;
|
||||
}
|
||||
|
||||
message GetContextByTypeAndNameResponse {
|
||||
optional Context context = 1;
|
||||
}
|
||||
|
||||
message GetContextsByIDRequest {
|
||||
// A list of context ids to retrieve.
|
||||
repeated int64 context_ids = 1;
|
||||
}
|
||||
|
||||
message GetContextsByIDResponse {
|
||||
// The result is not index-aligned: if an id is not found, it is not
|
||||
// returned.
|
||||
repeated Context contexts = 1;
|
||||
}
|
||||
|
||||
message GetContextsByArtifactRequest {
|
||||
optional int64 artifact_id = 1;
|
||||
}
|
||||
|
||||
message GetContextsByArtifactResponse {
|
||||
repeated Context contexts = 1;
|
||||
}
|
||||
|
||||
message GetContextsByExecutionRequest {
|
||||
optional int64 execution_id = 1;
|
||||
}
|
||||
|
||||
message GetContextsByExecutionResponse {
|
||||
repeated Context contexts = 1;
|
||||
}
|
||||
|
||||
message GetParentContextsByContextRequest {
|
||||
optional int64 context_id = 1;
|
||||
}
|
||||
|
||||
message GetParentContextsByContextResponse {
|
||||
repeated Context contexts = 1;
|
||||
}
|
||||
|
||||
message GetChildrenContextsByContextRequest {
|
||||
optional int64 context_id = 1;
|
||||
}
|
||||
|
||||
message GetChildrenContextsByContextResponse {
|
||||
repeated Context contexts = 1;
|
||||
}
|
||||
|
||||
message GetArtifactsByContextRequest {
|
||||
optional int64 context_id = 1;
|
||||
|
||||
// Specify List options.
|
||||
// Currently supports:
|
||||
// 1. Field to order the results.
|
||||
// 2. Page size.
|
||||
optional ListOperationOptions options = 2;
|
||||
}
|
||||
|
||||
message GetArtifactsByContextResponse {
|
||||
repeated Artifact artifacts = 1;
|
||||
|
||||
// Token to use to retreive next page of results if list options are used in
|
||||
// the request.
|
||||
optional string next_page_token = 2;
|
||||
}
|
||||
|
||||
message GetExecutionsByContextRequest {
|
||||
optional int64 context_id = 1;
|
||||
|
||||
// Specify List options.
|
||||
// Currently supports:
|
||||
// 1. Field to order the results.
|
||||
// 2. Page size.
|
||||
optional ListOperationOptions options = 2;
|
||||
}
|
||||
|
||||
message GetExecutionsByContextResponse {
|
||||
repeated Execution executions = 1;
|
||||
|
||||
// Token to use to retreive next page of results if list options are used in
|
||||
// the request.
|
||||
optional string next_page_token = 2;
|
||||
}
|
||||
|
||||
|
||||
// LINT.IfChange
|
||||
service MetadataStoreService {
|
||||
// Inserts or updates an ArtifactType.
|
||||
//
|
||||
// If no type exists in the database with the given name, it
|
||||
// creates a new type and returns the type_id.
|
||||
//
|
||||
// If the request type with the same name already exists (let's call it
|
||||
// stored_type), the method enforces the stored_type can be updated only when
|
||||
// the request type is backward compatible for the already stored instances.
|
||||
//
|
||||
// Backwards compatibility is violated iff:
|
||||
//
|
||||
// a) there is a property where the request type and stored_type have
|
||||
// different value type (e.g., int vs. string)
|
||||
// b) `can_add_fields = false` and the request type has a new property that
|
||||
// is not stored.
|
||||
// c) `can_omit_fields = false` and stored_type has an existing property
|
||||
// that is not provided in the request type.
|
||||
//
|
||||
// Args:
|
||||
// artifact_type: the type to be inserted or updated.
|
||||
// can_add_fields:
|
||||
// when set to true, new properties can be added;
|
||||
// when set to false, returns ALREADY_EXISTS if the request type has
|
||||
// properties that are not in stored_type.
|
||||
// can_omit_fields:
|
||||
// when set to true, stored properties can be omitted in the request type;
|
||||
// when set to false, returns ALREADY_EXISTS if the stored_type has
|
||||
// properties not in the request type.
|
||||
//
|
||||
// Returns:
|
||||
// The type_id of the stored type.
|
||||
//
|
||||
// Raises:
|
||||
// ALREADY_EXISTS error in the case listed above.
|
||||
// INVALID_ARGUMENT error, if the given type has no name, or any
|
||||
// property value type is unknown.
|
||||
rpc PutArtifactType(PutArtifactTypeRequest)
|
||||
returns (PutArtifactTypeResponse) {}
|
||||
|
||||
// Inserts or updates an ExecutionType. Please refer to PutArtifactType for
|
||||
// type upsert API description.
|
||||
rpc PutExecutionType(PutExecutionTypeRequest)
|
||||
returns (PutExecutionTypeResponse) {}
|
||||
|
||||
// Inserts or updates an ContextType. Please refer to PutArtifactType for
|
||||
// type upsert API description.
|
||||
rpc PutContextType(PutContextTypeRequest) returns (PutContextTypeResponse) {}
|
||||
|
||||
// Bulk inserts types atomically.
|
||||
rpc PutTypes(PutTypesRequest) returns (PutTypesResponse) {}
|
||||
|
||||
// Inserts or updates artifacts in the database.
|
||||
//
|
||||
// If an artifact_id is specified for an artifact, it is an update.
|
||||
// If an artifact_id is unspecified, it will insert a new artifact.
|
||||
// For new artifacts, type must be specified.
|
||||
// For old artifacts, type must be unchanged or unspecified.
|
||||
//
|
||||
// Args:
|
||||
// artifacts: A list of artifacts to insert or update.
|
||||
//
|
||||
// Returns:
|
||||
// A list of artifact ids index-aligned with the input.
|
||||
rpc PutArtifacts(PutArtifactsRequest) returns (PutArtifactsResponse) {}
|
||||
|
||||
// Inserts or updates executions in the database.
|
||||
//
|
||||
// If an execution_id is specified for an execution, it is an update.
|
||||
// If an execution_id is unspecified, it will insert a new execution.
|
||||
// For new executions, type must be specified.
|
||||
// For old executions, type must be unchanged or unspecified.
|
||||
//
|
||||
// Args:
|
||||
// executions: A list of executions to insert or update.
|
||||
//
|
||||
// Returns:
|
||||
// A list of execution ids index-aligned with the input.
|
||||
//
|
||||
rpc PutExecutions(PutExecutionsRequest) returns (PutExecutionsResponse) {}
|
||||
|
||||
// Inserts events in the database.
|
||||
//
|
||||
// The execution_id and artifact_id must already exist.
|
||||
// Once created, events cannot be modified.
|
||||
//
|
||||
// Args:
|
||||
// events: A list of events to insert or update.
|
||||
rpc PutEvents(PutEventsRequest) returns (PutEventsResponse) {}
|
||||
|
||||
// Inserts or updates an Execution and its input and output artifacts and
|
||||
// related contexts atomically. The `artifact_event_pairs` include the state
|
||||
// changes of the Artifacts used or generated by the Execution, as well as the
|
||||
// input/output Event. The `contexts` describe the associations of the
|
||||
// execution and the attributions of the artifacts.
|
||||
//
|
||||
// If an execution_id, artifact_id or context_id is specified, it is an
|
||||
// update, otherwise it does an insertion. For insertion, type must be
|
||||
// specified.
|
||||
//
|
||||
// Args:
|
||||
// execution: An execution to insert or update.
|
||||
// artifact_event_pairs: Artifacts to insert or update and events to insert.
|
||||
// contexts: The contexts that the execution and the artifacts belong to.
|
||||
//
|
||||
// Returns:
|
||||
// An execution id and a list of artifacts and contexts ids index-aligned
|
||||
// with the input.
|
||||
rpc PutExecution(PutExecutionRequest) returns (PutExecutionResponse) {}
|
||||
|
||||
// Inserts or updates contexts in database and returns a list of context ids.
|
||||
//
|
||||
// If an context_id is specified for a context, it is an update.
|
||||
// If an context_id is unspecified, it will insert a new context.
|
||||
// For new contexts, type must be specified.
|
||||
// For old contexts, type must be unchanged or unspecified.
|
||||
//
|
||||
// Args:
|
||||
// contexts: A list of contexts to insert or update.
|
||||
//
|
||||
// Returns:
|
||||
// A list of context ids index-aligned with the input.
|
||||
rpc PutContexts(PutContextsRequest) returns (PutContextsResponse) {}
|
||||
|
||||
// Inserts attribution and association relationships in the database.
|
||||
// The context_id, artifact_id, and execution_id must already exist.
|
||||
// If the relationship exists, this call does nothing. Once added, the
|
||||
// relationships cannot be modified.
|
||||
//
|
||||
// Args:
|
||||
// attributions: A list of attributions to insert.
|
||||
// associations: A list of associations to insert.
|
||||
rpc PutAttributionsAndAssociations(PutAttributionsAndAssociationsRequest)
|
||||
returns (PutAttributionsAndAssociationsResponse) {}
|
||||
|
||||
// Inserts parental context relationships in the database.
|
||||
// The ParentContext relationship has direction. The call fails if cycles are
|
||||
// detected.
|
||||
//
|
||||
// Args:
|
||||
// parent_contexts: A list of parent contexts to insert.
|
||||
rpc PutParentContexts(PutParentContextsRequest)
|
||||
returns (PutParentContextsResponse) {}
|
||||
|
||||
// Gets an artifact type. Returns a NOT_FOUND error if the type does not
|
||||
// exist.
|
||||
rpc GetArtifactType(GetArtifactTypeRequest)
|
||||
returns (GetArtifactTypeResponse) {}
|
||||
|
||||
// Gets a list of artifact types by ID.
|
||||
// If no artifact types with an ID exists, the artifact type is skipped.
|
||||
rpc GetArtifactTypesByID(GetArtifactTypesByIDRequest)
|
||||
returns (GetArtifactTypesByIDResponse) {}
|
||||
|
||||
// Gets a list of all artifact types.
|
||||
rpc GetArtifactTypes(GetArtifactTypesRequest)
|
||||
returns (GetArtifactTypesResponse) {}
|
||||
|
||||
// Gets an execution type, or None if it does not exist.
|
||||
rpc GetExecutionType(GetExecutionTypeRequest)
|
||||
returns (GetExecutionTypeResponse) {}
|
||||
|
||||
// Gets a list of execution types by ID.
|
||||
// If no execution types with an ID exists, the execution type is skipped.
|
||||
rpc GetExecutionTypesByID(GetExecutionTypesByIDRequest)
|
||||
returns (GetExecutionTypesByIDResponse) {}
|
||||
|
||||
// Gets a list of all execution types.
|
||||
rpc GetExecutionTypes(GetExecutionTypesRequest)
|
||||
returns (GetExecutionTypesResponse) {}
|
||||
|
||||
// Gets a context type. Returns a NOT_FOUND error if the type does not exist.
|
||||
rpc GetContextType(GetContextTypeRequest) returns (GetContextTypeResponse) {}
|
||||
|
||||
// Gets a list of context types by ID.
|
||||
// If no context types with an ID exists, the context type is skipped.
|
||||
rpc GetContextTypesByID(GetContextTypesByIDRequest)
|
||||
returns (GetContextTypesByIDResponse) {}
|
||||
|
||||
// Gets a list of all context types.
|
||||
rpc GetContextTypes(GetContextTypesRequest)
|
||||
returns (GetContextTypesResponse) {}
|
||||
|
||||
// Gets all the artifacts.
|
||||
rpc GetArtifacts(GetArtifactsRequest) returns (GetArtifactsResponse) {}
|
||||
|
||||
// Gets all the executions.
|
||||
rpc GetExecutions(GetExecutionsRequest) returns (GetExecutionsResponse) {}
|
||||
|
||||
// Gets all the contexts.
|
||||
rpc GetContexts(GetContextsRequest) returns (GetContextsResponse) {}
|
||||
|
||||
// Gets all artifacts with matching ids.
|
||||
//
|
||||
// The result is not index-aligned: if an id is not found, it is not returned.
|
||||
//
|
||||
// Args:
|
||||
// artifact_ids: A list of artifact ids to retrieve.
|
||||
//
|
||||
// Returns:
|
||||
// Artifacts with matching ids.
|
||||
rpc GetArtifactsByID(GetArtifactsByIDRequest)
|
||||
returns (GetArtifactsByIDResponse) {}
|
||||
|
||||
// Gets all executions with matching ids.
|
||||
//
|
||||
// The result is not index-aligned: if an id is not found, it is not returned.
|
||||
//
|
||||
// Args:
|
||||
// execution_ids: A list of execution ids to retrieve.
|
||||
rpc GetExecutionsByID(GetExecutionsByIDRequest)
|
||||
returns (GetExecutionsByIDResponse) {}
|
||||
|
||||
// Gets all contexts with matching ids.
|
||||
//
|
||||
// The result is not index-aligned: if an id is not found, it is not returned.
|
||||
//
|
||||
// Args:
|
||||
// context_ids: A list of context ids to retrieve.
|
||||
rpc GetContextsByID(GetContextsByIDRequest)
|
||||
returns (GetContextsByIDResponse) {}
|
||||
|
||||
// Gets all the artifacts of a given type.
|
||||
rpc GetArtifactsByType(GetArtifactsByTypeRequest)
|
||||
returns (GetArtifactsByTypeResponse) {}
|
||||
|
||||
// Gets all the executions of a given type.
|
||||
rpc GetExecutionsByType(GetExecutionsByTypeRequest)
|
||||
returns (GetExecutionsByTypeResponse) {}
|
||||
|
||||
// Gets all the contexts of a given type.
|
||||
rpc GetContextsByType(GetContextsByTypeRequest)
|
||||
returns (GetContextsByTypeResponse) {}
|
||||
|
||||
// Gets the artifact of the given type and artifact name.
|
||||
rpc GetArtifactByTypeAndName(GetArtifactByTypeAndNameRequest)
|
||||
returns (GetArtifactByTypeAndNameResponse) {}
|
||||
|
||||
// Gets the execution of the given type and execution name.
|
||||
rpc GetExecutionByTypeAndName(GetExecutionByTypeAndNameRequest)
|
||||
returns (GetExecutionByTypeAndNameResponse) {}
|
||||
|
||||
// Gets the context of the given type and context name.
|
||||
rpc GetContextByTypeAndName(GetContextByTypeAndNameRequest)
|
||||
returns (GetContextByTypeAndNameResponse) {}
|
||||
|
||||
// Gets all the artifacts with matching uris.
|
||||
rpc GetArtifactsByURI(GetArtifactsByURIRequest)
|
||||
returns (GetArtifactsByURIResponse) {}
|
||||
|
||||
// Gets all events with matching execution ids.
|
||||
rpc GetEventsByExecutionIDs(GetEventsByExecutionIDsRequest)
|
||||
returns (GetEventsByExecutionIDsResponse) {}
|
||||
|
||||
// Gets all events with matching artifact ids.
|
||||
rpc GetEventsByArtifactIDs(GetEventsByArtifactIDsRequest)
|
||||
returns (GetEventsByArtifactIDsResponse) {}
|
||||
|
||||
// Gets all context that an artifact is attributed to.
|
||||
rpc GetContextsByArtifact(GetContextsByArtifactRequest)
|
||||
returns (GetContextsByArtifactResponse) {}
|
||||
|
||||
// Gets all context that an execution is associated with.
|
||||
rpc GetContextsByExecution(GetContextsByExecutionRequest)
|
||||
returns (GetContextsByExecutionResponse) {}
|
||||
|
||||
// Gets all parent contexts that a context is related.
|
||||
rpc GetParentContextsByContext(GetParentContextsByContextRequest)
|
||||
returns (GetParentContextsByContextResponse) {}
|
||||
|
||||
// Gets all children contexts that a context is related.
|
||||
rpc GetChildrenContextsByContext(GetChildrenContextsByContextRequest)
|
||||
returns (GetChildrenContextsByContextResponse) {}
|
||||
|
||||
// Gets all direct artifacts that a context attributes to.
|
||||
rpc GetArtifactsByContext(GetArtifactsByContextRequest)
|
||||
returns (GetArtifactsByContextResponse) {}
|
||||
|
||||
// Gets all direct executions that a context associates with.
|
||||
rpc GetExecutionsByContext(GetExecutionsByContextRequest)
|
||||
returns (GetExecutionsByContextResponse) {}
|
||||
|
||||
}
|
||||
// LINT.ThenChange(../metadata_store/metadata_store_service_interface.h)
|
||||
Loading…
Reference in New Issue