TheNeuralBit commented on a change in pull request #12454:
URL: https://github.com/apache/beam/pull/12454#discussion_r464726158



##########
File path: 
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
##########
@@ -107,39 +107,39 @@
           ImmutableMap.builder();
       for (ExternalTransformRegistrar registrar :
           ServiceLoader.load(ExternalTransformRegistrar.class)) {
-        for (Map.Entry<String, Class<? extends ExternalTransformBuilder<?, ?, 
?>>> entry :
-            registrar.knownBuilders().entrySet()) {
+        for (Map.Entry<String, ExternalTransformBuilder<?, ?, ?>> entry :
+            registrar.knownBuilderInstances().entrySet()) {
           String urn = entry.getKey();
-          Class<? extends ExternalTransformBuilder<?, ?, ?>> builderClass = 
entry.getValue();
+          ExternalTransformBuilder builderInstance = entry.getValue();
           builder.put(
               urn,
               spec -> {
                 try {
                   ExternalTransforms.ExternalConfigurationPayload payload =
                       
ExternalTransforms.ExternalConfigurationPayload.parseFrom(spec.getPayload());
-                  return translate(payload, builderClass);
+                  return builderInstance.buildExternal(
+                      payloadToConfig(
+                          payload,
+                          (Class<? extends ExternalTransformBuilder<?, ?, ?>>)
+                              builderInstance.getClass()));
                 } catch (Exception e) {
                   throw new RuntimeException(
                       String.format("Failed to build transform %s from spec 
%s", urn, spec), e);
                 }
               });
         }
       }
+
       return builder.build();
     }
 
-    private static PTransform<?, ?> translate(
+    Object payloadToConfig(
         ExternalTransforms.ExternalConfigurationPayload payload,
         Class<? extends ExternalTransformBuilder<?, ?, ?>> builderClass)
         throws Exception {
-      Preconditions.checkState(
-          ExternalTransformBuilder.class.isAssignableFrom(builderClass),
-          "Provided identifier %s is not an ExternalTransformBuilder.",
-          builderClass.getName());
-
       Object configObject = initConfiguration(builderClass);
       populateConfiguration(configObject, payload);
-      return buildTransform(builderClass, configObject);

Review comment:
       I think after this change there are no callers to `buildTransform`, can 
you remove it?

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/expansion/ExternalTransformRegistrar.java
##########
@@ -31,4 +34,27 @@
 
   /** A mapping from URN to an {@link ExternalTransformBuilder} class. */
   Map<String, Class<? extends ExternalTransformBuilder<?, ?, ?>>> 
knownBuilders();

Review comment:
       There should be a default implementation for this that just returns an 
empty Map, that way new implementations will only have to override 
`knownBuilderInstances()`

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/expansion/ExternalTransformRegistrar.java
##########
@@ -31,4 +34,27 @@
 
   /** A mapping from URN to an {@link ExternalTransformBuilder} class. */

Review comment:
       Let's mark this Deprecated and we can remove it down the road.
   ```suggestion
     /** 
      * A mapping from URN to an {@link ExternalTransformBuilder} class.
      *
      * @deprecated Prefer implementing `knownBuilderInstances`. This method 
will be removed in a future version of Beam.
      */
      @Deprecated
   ```




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to