chamikaramj commented on code in PR #30808:
URL: https://github.com/apache/beam/pull/30808#discussion_r1552590182


##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.managed;
+
+import com.google.auto.value.AutoValue;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.utils.YamlUtils;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+
+public class Managed {
+  public static final String ICEBERG = "iceberg";
+
+  public static Read read(String source) {
+
+    return new AutoValue_Managed_Read.Builder()
+        .setSource(
+            Preconditions.checkNotNull(
+                Read.TRANSFORMS.get(source.toLowerCase()),
+                "An unsupported source was specified: '%s'. Please specify one 
of the following source: %s",
+                source,
+                Read.TRANSFORMS.keySet()))
+        .setSupportedIdentifiers(new ArrayList<>(Read.TRANSFORMS.values()))
+        .build();
+  }
+
+  @AutoValue
+  public abstract static class Read extends SchemaTransform {
+    public static final Map<String, String> TRANSFORMS =
+        ImmutableMap.<String, String>builder()
+            .put(ICEBERG, 
"beam:schematransform:org.apache.beam:iceberg_read:v1")
+            .build();
+
+    abstract String getSource();
+
+    abstract @Nullable String getConfig();
+
+    abstract @Nullable String getConfigUrl();
+
+    abstract List<String> getSupportedIdentifiers();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSource(String source);
+
+      abstract Builder setConfig(@Nullable String config);
+
+      abstract Builder setConfigUrl(@Nullable String configUrl);
+
+      abstract Builder setSupportedIdentifiers(List<String> 
supportedIdentifiers);
+
+      abstract Read build();
+    }
+
+    public Read withConfigUrl(String configUrl) {

Review Comment:
   Please add Java docs for `withConfig` methods.



##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.managed;
+
+import com.google.auto.value.AutoValue;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.utils.YamlUtils;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+
+public class Managed {
+  public static final String ICEBERG = "iceberg";
+
+  public static Read read(String source) {
+
+    return new AutoValue_Managed_Read.Builder()
+        .setSource(
+            Preconditions.checkNotNull(
+                Read.TRANSFORMS.get(source.toLowerCase()),
+                "An unsupported source was specified: '%s'. Please specify one 
of the following source: %s",
+                source,
+                Read.TRANSFORMS.keySet()))
+        .setSupportedIdentifiers(new ArrayList<>(Read.TRANSFORMS.values()))
+        .build();
+  }
+
+  @AutoValue
+  public abstract static class Read extends SchemaTransform {
+    public static final Map<String, String> TRANSFORMS =
+        ImmutableMap.<String, String>builder()
+            .put(ICEBERG, 
"beam:schematransform:org.apache.beam:iceberg_read:v1")
+            .build();
+
+    abstract String getSource();
+
+    abstract @Nullable String getConfig();
+
+    abstract @Nullable String getConfigUrl();
+
+    abstract List<String> getSupportedIdentifiers();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSource(String source);
+
+      abstract Builder setConfig(@Nullable String config);
+
+      abstract Builder setConfigUrl(@Nullable String configUrl);
+
+      abstract Builder setSupportedIdentifiers(List<String> 
supportedIdentifiers);
+
+      abstract Read build();
+    }
+
+    public Read withConfigUrl(String configUrl) {
+      return toBuilder().setConfigUrl(configUrl).build();
+    }
+
+    public Read withConfig(String config) {
+      return toBuilder().setConfig(config).build();
+    }
+
+    public Read withConfig(Map<String, Object> config) {
+      return 
toBuilder().setConfig(YamlUtils.yamlStringFromMap(config)).build();
+    }
+
+    @VisibleForTesting
+    Read withSupportedIdentifiers(List<String> supportedIdentifiers) {
+      return toBuilder().setSupportedIdentifiers(supportedIdentifiers).build();
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      ManagedSchemaTransformProvider.ManagedConfig managedConfig =
+          ManagedSchemaTransformProvider.ManagedConfig.builder()
+              .setTransformIdentifier(getSource())
+              .setConfig(getConfig())
+              .setConfigUrl(getConfigUrl())
+              .build();
+
+      SchemaTransform underlyingTransform =
+          
ManagedSchemaTransformProvider.of(TRANSFORMS.values()).from(managedConfig);
+
+      return input.apply(underlyingTransform);
+    }
+  }
+
+  public static Write write(String sink) {

Review Comment:
   Ditto regarding Java docs.



##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/ManagedSchemaTransformProvider.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.managed;
+
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.utils.YamlUtils;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
+
+@AutoService(SchemaTransformProvider.class)
+class ManagedSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<ManagedSchemaTransformProvider.ManagedConfig> {
+
+  @Override
+  public String identifier() {
+    return "beam:schematransform:org.apache.beam:managed:v1";
+  }
+
+  private final Map<String, SchemaTransformProvider> schemaTransformProviders 
= new HashMap<>();
+
+  private ManagedSchemaTransformProvider(Collection<String> identifiers) {
+    try {
+      for (SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(SchemaTransformProvider.class)) {
+        if 
(schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the 
same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), 
schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+
+    schemaTransformProviders.entrySet().removeIf(e -> 
!identifiers.contains(e.getKey()));
+  }
+
+  private static @Nullable ManagedSchemaTransformProvider managedProvider = 
null;
+
+  public static ManagedSchemaTransformProvider of(Collection<String> 
supportedIdentifiers) {
+    if (managedProvider == null) {
+      managedProvider = new 
ManagedSchemaTransformProvider(supportedIdentifiers);
+    }
+    return managedProvider;
+  }
+
+  @DefaultSchema(AutoValueSchema.class)
+  @AutoValue
+  public abstract static class ManagedConfig {

Review Comment:
   Please make this class and methods package private.



##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.managed;
+
+import com.google.auto.value.AutoValue;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.utils.YamlUtils;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+
+public class Managed {
+  public static final String ICEBERG = "iceberg";
+
+  public static Read read(String source) {

Review Comment:
   Please add Java docs and clarify what can be specified as the source by 
explicitly listing them (and refer to the constants defined above).



##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.managed;
+
+import com.google.auto.value.AutoValue;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.utils.YamlUtils;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+
+public class Managed {
+  public static final String ICEBERG = "iceberg";

Review Comment:
   Add a comment that this will be moved into a dynamically generated list in 
the future.



##########
sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.managed;
+
+import com.google.auto.value.AutoValue;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.utils.YamlUtils;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+
+public class Managed {
+  public static final String ICEBERG = "iceberg";
+
+  public static Read read(String source) {
+
+    return new AutoValue_Managed_Read.Builder()
+        .setSource(
+            Preconditions.checkNotNull(
+                Read.TRANSFORMS.get(source.toLowerCase()),
+                "An unsupported source was specified: '%s'. Please specify one 
of the following source: %s",
+                source,
+                Read.TRANSFORMS.keySet()))
+        .setSupportedIdentifiers(new ArrayList<>(Read.TRANSFORMS.values()))
+        .build();
+  }
+
+  @AutoValue
+  public abstract static class Read extends SchemaTransform {
+    public static final Map<String, String> TRANSFORMS =
+        ImmutableMap.<String, String>builder()
+            .put(ICEBERG, 
"beam:schematransform:org.apache.beam:iceberg_read:v1")
+            .build();
+
+    abstract String getSource();
+
+    abstract @Nullable String getConfig();
+
+    abstract @Nullable String getConfigUrl();
+
+    abstract List<String> getSupportedIdentifiers();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSource(String source);
+
+      abstract Builder setConfig(@Nullable String config);
+
+      abstract Builder setConfigUrl(@Nullable String configUrl);
+
+      abstract Builder setSupportedIdentifiers(List<String> 
supportedIdentifiers);
+
+      abstract Read build();
+    }
+
+    public Read withConfigUrl(String configUrl) {
+      return toBuilder().setConfigUrl(configUrl).build();
+    }
+
+    public Read withConfig(String config) {
+      return toBuilder().setConfig(config).build();
+    }
+
+    public Read withConfig(Map<String, Object> config) {
+      return 
toBuilder().setConfig(YamlUtils.yamlStringFromMap(config)).build();
+    }
+
+    @VisibleForTesting
+    Read withSupportedIdentifiers(List<String> supportedIdentifiers) {
+      return toBuilder().setSupportedIdentifiers(supportedIdentifiers).build();
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      ManagedSchemaTransformProvider.ManagedConfig managedConfig =
+          ManagedSchemaTransformProvider.ManagedConfig.builder()
+              .setTransformIdentifier(getSource())
+              .setConfig(getConfig())
+              .setConfigUrl(getConfigUrl())
+              .build();
+
+      SchemaTransform underlyingTransform =
+          
ManagedSchemaTransformProvider.of(TRANSFORMS.values()).from(managedConfig);
+
+      return input.apply(underlyingTransform);
+    }
+  }
+
+  public static Write write(String sink) {
+    return new AutoValue_Managed_Write.Builder()
+        .setSink(
+            Preconditions.checkNotNull(
+                Write.TRANSFORMS.get(sink.toLowerCase()),
+                "An unsupported sink was specified: '%s'. Please specify one 
of the following sinks: %s",
+                sink,
+                Write.TRANSFORMS.keySet()))
+        .setSupportedIdentifiers(new ArrayList<>(Write.TRANSFORMS.values()))
+        .build();
+  }
+
+  @AutoValue
+  public abstract static class Write extends SchemaTransform {
+    public static final Map<String, String> TRANSFORMS =
+        ImmutableMap.<String, String>builder()
+            .put(ICEBERG, 
"beam:schematransform:org.apache.beam:iceberg_write:v1")
+            .build();
+
+    abstract String getSink();
+
+    abstract @Nullable String getConfig();
+
+    abstract @Nullable String getConfigUrl();
+
+    abstract List<String> getSupportedIdentifiers();
+
+    abstract Builder toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setSink(String source);
+
+      abstract Builder setConfig(@Nullable String config);
+
+      abstract Builder setConfigUrl(@Nullable String configUrl);
+
+      abstract Builder setSupportedIdentifiers(List<String> 
supportedIdentifiers);
+
+      abstract Write build();
+    }
+
+    public Write withConfigUrl(String configUrl) {

Review Comment:
   Ditto.



##########
sdks/java/managed/src/test/java/org/apache/beam/sdk/managed/ManagedTest.java:
##########
@@ -0,0 +1,2 @@
+package org.apache.beam.sdk.managed;public class ManagedTest {
+}

Review Comment:
   Add tests ?



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