| /* |
| * Licensed to the Apache Software Foundation (ASF) under one |
| * or more contributor license agreements. See the NOTICE file |
| * distributed with this work for additional information |
| * regarding copyright ownership. The ASF licenses this file |
| * to you 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. |
| */ |
| |
| /* |
| * Protocol Buffers describing the external transforms available. |
| */ |
| |
| syntax = "proto3"; |
| |
| package org.apache.beam.model.pipeline.v1; |
| |
| option go_package = "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1;pipeline_v1"; |
| option java_package = "org.apache.beam.model.pipeline.v1"; |
| option java_outer_classname = "ExternalTransforms"; |
| |
| import "schema.proto"; |
| import "beam_runner_api.proto"; |
| |
| // A configuration payload for an external transform. |
| // Used as the payload of ExternalTransform as part of an ExpansionRequest. |
| message ExternalConfigurationPayload { |
| // A schema for use in beam:coder:row:v1 |
| Schema schema = 1; |
| |
| // A payload which can be decoded using beam:coder:row:v1 and the given |
| // schema. |
| bytes payload = 2; |
| } |
| |
| // Defines specific expansion methods that may be used to expand cross-language |
| // transforms. |
| // Has to be set as the URN of the transform of the expansion request. |
| message ExpansionMethods { |
| enum Enum { |
| // Expand a Java transform using specified constructor and builder methods. |
| // Transform payload will be of type JavaClassLookupPayload. |
| JAVA_CLASS_LOOKUP = 0 [(org.apache.beam.model.pipeline.v1.beam_urn) = |
| "beam:expansion:payload:java_class_lookup:v1"]; |
| } |
| } |
| |
| // A configuration payload for an external transform. |
| // Used to define a Java transform that can be directly instantiated by a Java |
| // expansion service. |
| message JavaClassLookupPayload { |
| // Name of the Java transform class. |
| string class_name = 1; |
| |
| // A static method to construct the initial instance of the transform. |
| // If not provided, the transform should be instantiated using a class |
| // constructor. |
| string constructor_method = 2; |
| |
| // The top level fields of the schema represent the method parameters in |
| // order. |
| // If able, top level field names are also verified against the method |
| // parameters for a match. |
| // Any field names in the form 'ignore[0-9]+' will not be used for validation |
| // hence that format can be used to represent arbitrary field names. |
| Schema constructor_schema = 3; |
| |
| // A payload which can be decoded using beam:coder:row:v1 and the provided |
| // constructor schema. |
| bytes constructor_payload = 4; |
| |
| // Set of builder methods and corresponding parameters to apply after the |
| // transform object is constructed. |
| // When constructing the transform object, given builder methods will be |
| // applied in order. |
| repeated BuilderMethod builder_methods = 5; |
| } |
| |
| // This represents a builder method of the transform class that should be |
| // applied in-order after instantiating the initial transform object. |
| // Each builder method may take one or more parameters and has to return an |
| // instance of the transform object. |
| message BuilderMethod { |
| // Name of the builder method |
| string name = 1; |
| |
| // The top level fields of the schema represent the method parameters in |
| // order. |
| // If able, top level field names are also verified against the method |
| // parameters for a match. |
| // Any field names in the form 'ignore[0-9]+' will not be used for validation |
| // hence that format can be used to represent arbitrary field names. |
| Schema schema = 2; |
| |
| // A payload which can be decoded using beam:coder:row:v1 and the builder |
| // method schema. |
| bytes payload = 3; |
| } |
| |
| |