chamikaramj commented on code in PR #30808: URL: https://github.com/apache/beam/pull/30808#discussion_r1554614494
########## sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java: ########## @@ -0,0 +1,253 @@ +/* + * 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.transforms.SchemaTransformProvider; +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; + +/** + * Top-level {@link org.apache.beam.sdk.transforms.PTransform}s that build and instantiate turnkey + * transforms. + * + * <h3>Available transforms</h3> + * + * <p>This API currently supports two operations: {@link Read} and {@link Write}. Each one + * enumerates the available transforms in a {@code TRANSFORMS} map. + * + * <h3>Building a Managed turnkey transform</h3> + * + * <p>Turnkey transforms are represented as {@link SchemaTransform}s, which means each one has a + * defined configuration. A given transform can be built with a {@code Map<String, Object>} that + * specifies arguments using like so: + * + * <pre>{@code + * PCollectionRowTuple output = Managed.read(ICEBERG) + * .withConfig(ImmutableMap.<String, Map>.builder() + * .put("foo", "abc") + * .put("bar", 123) + * .build()); + * }</pre> + * + * <p>Instead of specifying configuration arguments directly in the code, one can provide the + * location to a YAML file that contains this information. Say we have the following YAML file: + * + * <pre>{@code + * foo: "abc" + * bar: 123 + * }</pre> + * + * <p>The file's path can be passed in to the Managed API like so: + * + * <pre>{@code + * PCollectionRowTuple output = Managed.write(ICEBERG) + * .withConfigUrl(<config path>); + * }</pre> + */ +public class Managed { + + // TODO: Dynamically generate a list of supported transforms + public static final String ICEBERG = "iceberg"; + + /** + * Instantiates a {@link Managed.Read} transform for the specified source. The supported managed + * sources are: + * + * <ul> + * <li>{@link Managed#ICEBERG} : Read from Apache Iceberg + * </ul> + */ + 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 sources: %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 sourceIdentifier); + + abstract Builder setConfig(@Nullable String config); + + abstract Builder setConfigUrl(@Nullable String configUrl); + + abstract Builder setSupportedIdentifiers(List<String> supportedIdentifiers); + + abstract Read build(); + } + + /** + * Use the input Map of configuration arguments to build and instantiate the underlying + * transform. The map can ignore nullable parameters, but needs to include all required + * parameters. Check the underlying transform's schema ({@link + * SchemaTransformProvider#configurationSchema()}) to see which parameters are available. + */ + public Read withConfig(Map<String, Object> config) { + return toBuilder().setConfig(YamlUtils.yamlStringFromMap(config)).build(); + } + + /** + * Like {@link #withConfig(Map)}, but instead extracts the configuration arguments from a + * specified YAML file location. + */ + public Read withConfigUrl(String configUrl) { + return toBuilder().setConfigUrl(configUrl).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 = + new ManagedSchemaTransformProvider(getSupportedIdentifiers()).from(managedConfig); + + return input.apply(underlyingTransform); + } + } + + /** + * Instantiates a {@link Managed.Write} transform for the specified sink. The supported managed + * sinks are: + * + * <ul> + * <li>{@link Managed#ICEBERG} : Write to Apache Iceberg + * </ul> + */ + 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 sinkIdentifier); + + abstract Builder setConfig(@Nullable String config); + + abstract Builder setConfigUrl(@Nullable String configUrl); + + abstract Builder setSupportedIdentifiers(List<String> supportedIdentifiers); + + abstract Write build(); + } + + /** + * Use the input Map of configuration arguments to build and instantiate the underlying sink. + * The map can ignore nullable parameters, but needs to include all required parameters. Check + * the underlying sink's configuration schema to see which parameters are available. + */ + public Write withConfig(Map<String, Object> config) { + return toBuilder().setConfig(YamlUtils.yamlStringFromMap(config)).build(); + } + + /** + * Like {@link #withConfig(Map)}, but instead extracts the configuration arguments from a + * specified YAML file location. + */ + public Write withConfigUrl(String configUrl) { + return toBuilder().setConfigUrl(configUrl).build(); + } + + @VisibleForTesting + Write withSupportedIdentifiers(List<String> supportedIdentifiers) { + return toBuilder().setSupportedIdentifiers(supportedIdentifiers).build(); + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + ManagedSchemaTransformProvider.ManagedConfig managedConfig = + ManagedSchemaTransformProvider.ManagedConfig.builder() Review Comment: And will return a single class `ManagedTransform` instead of `Read` and `Write`. `ManagedTransform managedTransform = Managed.read(ICEBERG).witnConfig(configMap)` -- 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]
