ihji commented on code in PR #17280:
URL: https://github.com/apache/beam/pull/17280#discussion_r852359639


##########
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java:
##########
@@ -126,12 +127,33 @@ PCollection<OutputT> toOutputCollection(Map<TupleTag<?>, 
PCollection> output) {
 
     public MultiOutputExpandableTransform<InputT> withMultiOutputs() {
       return new MultiOutputExpandableTransform<>(
-          getUrn(), getPayload(), getEndpoint(), getClientFactory(), 
getNamespaceIndex());
+          getUrn(),
+          getPayload(),
+          getEndpoint(),
+          getClientFactory(),
+          getNamespaceIndex(),
+          getOutputCoders());
     }
 
-    public <T> SingleOutputExpandableTransform<InputT, T> withOutputType() {
+    public SingleOutputExpandableTransform<InputT, OutputT> 
withOutputCoder(Coder outputCoder) {
       return new SingleOutputExpandableTransform<>(
-          getUrn(), getPayload(), getEndpoint(), getClientFactory(), 
getNamespaceIndex());
+          getUrn(),
+          getPayload(),
+          getEndpoint(),
+          getClientFactory(),
+          getNamespaceIndex(),
+          ImmutableMap.of("0", outputCoder));
+    }
+
+    public SingleOutputExpandableTransform<InputT, OutputT> withOutputCoder(

Review Comment:
   Ah, good catch. It doesn't make sense on single output transform. Moved to 
multi output variant.



##########
model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto:
##########
@@ -46,6 +46,12 @@ message ExpansionRequest {
   // A namespace (prefix) to use for the id of any newly created
   // components.
   string namespace = 3;
+
+  // (Optional) Map from a local output tag to a coder id.
+  // If it is set, asks the expansion service to use the given
+  // coders for the output PCollections. Note that the request
+  // may not be fulfilled.
+  map<string, string> output_coder_override = 4;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to