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



##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProvider.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.schemas.transforms;
+
+import java.util.List;
+import java.util.Optional;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * Like {@link SchemaTransformProvider} except uses a configuration object 
instead of Schema and
+ * Row.
+ *
+ * <p>ConfigT should be available in the SchemaRegistry.
+ *
+ * <p><b>Internal only:</b> This interface is actively being worked on and it 
will likely change as
+ * we provide implementations for more standard Beam transforms. We provide no 
backwards
+ * compatibility guarantees and it should not be implemented outside of the 
Beam repository.
+ */
+@Internal
+@Experimental(Kind.SCHEMAS)
+public abstract class TypedSchemaTransformProvider<ConfigT> implements 
SchemaTransformProvider {
+
+  abstract Class<ConfigT> configurationClass();
+
+  /**
+   * Produce a SchemaTransform from ConfigT. Can throw a {@link 
InvalidConfigurationException} or a
+   * {@link InvalidSchemaException}.
+   */
+  abstract SchemaTransform from(ConfigT configuration);
+
+  /**
+   * List the dependencies needed for this transform. Jars from classpath are 
used by default when
+   * Optional.empty() is returned.
+   */
+  Optional<List<String>> dependencies(ConfigT configuration, PipelineOptions 
options) {
+    return Optional.empty();
+  }
+
+  @Override
+  public Schema configurationSchema() {
+    try {
+      return SchemaRegistry.createDefault().getSchema(configurationClass());
+    } catch (NoSuchSchemaException e) {
+      throw new RuntimeException(
+          "Unable to find schema for "
+              + identifier()
+              + " SchemaTransformProvider's configuration.");
+    }
+  }
+
+  @Override
+  public SchemaTransform from(Row configuration) {
+    return from(configFromRow(configuration));
+  }
+
+  @Override
+  public Optional<List<String>> dependencies(Row configuration, 
PipelineOptions options) {
+    return dependencies(configFromRow(configuration), options);
+  }

Review comment:
       nit: Let's make these final so implementors can only override the typed 
alternatives
   ```suggestion
     @Override
     public final Schema configurationSchema() {
       try {
         return SchemaRegistry.createDefault().getSchema(configurationClass());
       } catch (NoSuchSchemaException e) {
         throw new RuntimeException(
             "Unable to find schema for "
                 + identifier()
                 + " SchemaTransformProvider's configuration.");
       }
     }
   
     @Override
     public final SchemaTransform from(Row configuration) {
       return from(configFromRow(configuration));
     }
   
     @Override
     public final Optional<List<String>> dependencies(Row configuration, 
PipelineOptions options) {
       return dependencies(configFromRow(configuration), options);
     }
   ```
   




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