[ 
https://issues.apache.org/jira/browse/BEAM-6730?focusedWorklogId=214651&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-214651
 ]

ASF GitHub Bot logged work on BEAM-6730:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Mar/19 09:10
            Start Date: 18/Mar/19 09:10
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on pull request #7875: [BEAM-6730] 
Support configuring transforms externally in Java SDK / Expose Java's 
GenerateSequence in Python
URL: https://github.com/apache/beam/pull/7875#discussion_r266343725
 
 

 ##########
 File path: 
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionService.java
 ##########
 @@ -69,14 +82,123 @@
     Map<String, TransformProvider> knownTransforms();
   }
 
+  /**
+   * Exposes Java transforms via {@link 
org.apache.beam.sdk.expansion.ExternalTransformRegistrar}.
+   */
+  @AutoService(ExpansionService.ExpansionServiceRegistrar.class)
+  public static class ExternalTransformRegistrarLoader<ConfigT>
+      implements ExpansionService.ExpansionServiceRegistrar {
+
+    @Override
+    public Map<String, ExpansionService.TransformProvider> knownTransforms() {
+      ImmutableMap.Builder<String, ExpansionService.TransformProvider> builder 
=
+          ImmutableMap.builder();
+      for (ExternalTransformRegistrar registrar :
+          ServiceLoader.load(ExternalTransformRegistrar.class)) {
+        for (Map.Entry<String, Class<? extends ExternalTransformBuilder<?, ?, 
?>>> entry :
+            registrar.knownBuilders().entrySet()) {
+          String urn = entry.getKey();
+          Class<? extends ExternalTransformBuilder<?, ?, ?>> builderClass = 
entry.getValue();
+          builder.put(
+              urn,
+              spec -> {
+                try {
+                  ExternalTransforms.ExternalConfigurationPayload payload =
+                      
ExternalTransforms.ExternalConfigurationPayload.parseFrom(spec.getPayload());
+                  return translate(payload, builderClass);
+                } 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(
+        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);
+    }
+
+    private static Object initConfiguration(
+        Class<? extends ExternalTransformBuilder<?, ?, ?>> builderClass) 
throws Exception {
+      for (Method method : builderClass.getMethods()) {
+        if (method.getName().equals("buildExternal")) {
+          Preconditions.checkState(
+              method.getParameterCount() == 1,
+              "Build method for ExternalTransformBuilder %s must have exactly 
one parameter, but had %s parameters.",
+              builderClass.getSimpleName(),
+              method.getParameterCount());
+          Class<?> configurationClass = method.getParameterTypes()[0];
+          if (Object.class.equals(configurationClass)) {
+            continue;
+          }
+          return configurationClass.getDeclaredConstructor().newInstance();
+        }
+      }
+      throw new RuntimeException("Couldn't find build method on 
ExternalTransformBuilder.");
+    }
+
+    private static void populateConfiguration(
+        Object config, ExternalTransforms.ExternalConfigurationPayload 
payload) throws Exception {
+      Converter<String, String> camelCaseConverter =
+          CaseFormat.LOWER_UNDERSCORE.converterTo(CaseFormat.LOWER_CAMEL);
+      for (Map.Entry<String, ExternalTransforms.ConfigValue> entry :
+          payload.getConfigurationMap().entrySet()) {
+        String fieldName = camelCaseConverter.convert(entry.getKey());
+        String coderUrn = entry.getValue().getCoderUrn();
+        final Coder coder;
+        if 
(BeamUrns.getUrn(RunnerApi.StandardCoders.Enum.VARINT).equals(coderUrn)) {
 
 Review comment:
   Shouldn't we be leveraging the coder rehydration code here? Or is that 
implicit in the TODO below (which should be made more explicit)? 
 
----------------------------------------------------------------
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:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 214651)
    Time Spent: 9.5h  (was: 9h 20m)

> Expose Java transforms (specifically IO) in other SDKs
> ------------------------------------------------------
>
>                 Key: BEAM-6730
>                 URL: https://issues.apache.org/jira/browse/BEAM-6730
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-flink, sdk-java-core, sdk-py-core
>            Reporter: Maximilian Michels
>            Assignee: Maximilian Michels
>            Priority: Major
>          Time Spent: 9.5h
>  Remaining Estimate: 0h
>
> Since https://github.com/apache/beam/pull/7316 we can reference external 
> transforms which are transforms only available in a "foreign" SDKs. This 
> allows us to fill the gap in terms of missing transforms in the Python and Go 
> SDK, specifically IO transforms.
> We can start collecting/exposing transforms that Beam users need. The 
> following transforms could be interesting:
> - KafkaIO / KinesisIO
> - CassandraIO / ElasticserchIO / Hbase / Redis
> - JDBC
> - S3 file system
> - GenerateSequence
> See also https://s.apache.org/beam-cross-language-io and BEAM-6485.
> CC [~robertwb] [~chamikara] [~thw]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to