You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/14 14:59:25 UTC

[GitHub] [beam] johnjcasey commented on a diff in pull request #21765: [BEAM-14073] [CdapIO] CDAP IO for batch plugins: Read, Write. Unit tests included

johnjcasey commented on code in PR #21765:
URL: https://github.com/apache/beam/pull/21765#discussion_r896908439


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -630,6 +631,7 @@ class BeamModulePlugin implements Plugin<Project> {
         guava_testlib                               : "com.google.guava:guava-testlib:$guava_version",
         hadoop_client                               : "org.apache.hadoop:hadoop-client:$hadoop_version",
         hadoop_common                               : "org.apache.hadoop:hadoop-common:$hadoop_version",
+        hadoop_mapred                               : "org.apache.hadoop:hadoop-mapred:0.22.0",

Review Comment:
   should this be $hadoop_version instead of a hardcoded value?



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.io.cdap;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import io.cdap.cdap.api.plugin.PluginConfig;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.hadoop.format.HDFSSynchronization;
+import org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.hadoop.conf.Configuration;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * An unbounded/bounded sources and sinks from <a
+ * href="https://github.com/data-integrations">CDAP</a> plugins.
+ */
+@SuppressWarnings({
+  "argument.type.incompatible",
+  "return.type.incompatible",
+  "dereference.of.nullable",
+  "UnnecessaryParentheses"
+})
+public class CdapIO {
+
+  public static <K, V> Read<K, V> read() {
+    return new AutoValue_CdapIO_Read.Builder<K, V>().build();
+  }
+
+  public static <K, V> Write<K, V> write() {
+    return new AutoValue_CdapIO_Write.Builder<K, V>().build();
+  }
+
+  /** A {@link PTransform} to read from CDAP source. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<K, V> extends PTransform<PBegin, PCollection<KV<K, V>>> {
+
+    abstract @Nullable PluginConfig getPluginConfig();
+
+    abstract @Nullable Plugin getCdapPlugin();
+
+    abstract @Nullable Class<K> getKeyClass();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract Builder<K, V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)
+    @AutoValue.Builder
+    abstract static class Builder<K, V> {
+
+      abstract Builder<K, V> setPluginConfig(PluginConfig config);
+
+      abstract Builder<K, V> setCdapPlugin(Plugin plugin);
+
+      abstract Builder<K, V> setKeyClass(Class<K> keyClass);
+
+      abstract Builder<K, V> setValueClass(Class<V> valueClass);
+
+      abstract Read<K, V> build();
+    }
+
+    public Read<K, V> withCdapPlugin(Plugin plugin) {
+      checkArgument(plugin != null, "Cdap plugin can not be null");

Review Comment:
   by convention, we are using Preconditions.checkNotNull for these type of checks



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.io.cdap;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import io.cdap.cdap.api.plugin.PluginConfig;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.hadoop.format.HDFSSynchronization;
+import org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.hadoop.conf.Configuration;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * An unbounded/bounded sources and sinks from <a
+ * href="https://github.com/data-integrations">CDAP</a> plugins.
+ */
+@SuppressWarnings({
+  "argument.type.incompatible",
+  "return.type.incompatible",
+  "dereference.of.nullable",
+  "UnnecessaryParentheses"
+})
+public class CdapIO {
+
+  public static <K, V> Read<K, V> read() {
+    return new AutoValue_CdapIO_Read.Builder<K, V>().build();
+  }
+
+  public static <K, V> Write<K, V> write() {
+    return new AutoValue_CdapIO_Write.Builder<K, V>().build();
+  }
+
+  /** A {@link PTransform} to read from CDAP source. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<K, V> extends PTransform<PBegin, PCollection<KV<K, V>>> {
+
+    abstract @Nullable PluginConfig getPluginConfig();
+
+    abstract @Nullable Plugin getCdapPlugin();
+
+    abstract @Nullable Class<K> getKeyClass();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract Builder<K, V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)
+    @AutoValue.Builder
+    abstract static class Builder<K, V> {
+
+      abstract Builder<K, V> setPluginConfig(PluginConfig config);
+
+      abstract Builder<K, V> setCdapPlugin(Plugin plugin);
+
+      abstract Builder<K, V> setKeyClass(Class<K> keyClass);
+
+      abstract Builder<K, V> setValueClass(Class<V> valueClass);
+
+      abstract Read<K, V> build();
+    }
+
+    public Read<K, V> withCdapPlugin(Plugin plugin) {
+      checkArgument(plugin != null, "Cdap plugin can not be null");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Read<K, V> withCdapPluginClass(Class<?> cdapPluginClass) {
+      checkArgument(cdapPluginClass != null, "Cdap plugin class can not be null");
+      Plugin plugin = MappingUtils.getPluginByClass(cdapPluginClass);
+      checkArgument(plugin != null, "Can not instantiate Plugin object from provided plugin class");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Read<K, V> withPluginConfig(PluginConfig pluginConfig) {
+      checkArgument(pluginConfig != null, "Plugin config can not be null");
+      return toBuilder().setPluginConfig(pluginConfig).build();
+    }
+
+    public Read<K, V> withKeyClass(Class<K> keyClass) {
+      checkArgument(keyClass != null, "Key class can not be null");
+      return toBuilder().setKeyClass(keyClass).build();
+    }
+
+    public Read<K, V> withValueClass(Class<V> valueClass) {
+      checkArgument(valueClass != null, "Value class can not be null");
+      return toBuilder().setValueClass(valueClass).build();
+    }
+
+    @Override
+    public PCollection<KV<K, V>> expand(PBegin input) {
+      validateTransform();
+
+      getCdapPlugin()
+          .withConfig(getPluginConfig())
+          .withHadoopConfiguration(getKeyClass(), getValueClass())
+          .prepareRun();
+
+      if (getCdapPlugin().isUnbounded()) {
+        // TODO: implement SparkReceiverIO.<~>read()

Review Comment:
   This should probably throw an exception for now, to prevent confusion



##########
sdks/java/io/cdap/build.gradle:
##########
@@ -38,26 +38,35 @@ interface for integration with CDAP plugins."""
  */
 
 dependencies {
-    implementation library.java.avro
     implementation library.java.cdap_api
     implementation library.java.cdap_api_commons
     implementation (library.java.cdap_common) {
-        exclude module: "log4j-over-slf4j"
+        exclude module: 'log4j-over-slf4j'
     }
     implementation library.java.cdap_etl_api
     implementation library.java.cdap_etl_api_spark
+    implementation library.java.cdap_hydrator_common
+    implementation library.java.guava
     implementation library.java.hadoop_common
-    implementation library.java.hadoop_mapreduce_client_core
+    implementation library.java.hadoop_mapred
     implementation library.java.jackson_core
     implementation library.java.jackson_databind
-    implementation library.java.guava
     implementation library.java.slf4j_api
     implementation library.java.tephra
+    implementation library.java.vendored_guava_26_0_jre
     implementation project(path: ":sdks:java:core", configuration: "shadow")
+    implementation project(":sdks:java:io:hadoop-format")
+    implementation "io.cdap.plugin:salesforce-plugins:1.4.0"

Review Comment:
   should these be moved up to the beam module plugin?



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.io.cdap;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import io.cdap.cdap.api.plugin.PluginConfig;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.hadoop.format.HDFSSynchronization;
+import org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.hadoop.conf.Configuration;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * An unbounded/bounded sources and sinks from <a
+ * href="https://github.com/data-integrations">CDAP</a> plugins.
+ */
+@SuppressWarnings({
+  "argument.type.incompatible",
+  "return.type.incompatible",
+  "dereference.of.nullable",
+  "UnnecessaryParentheses"
+})
+public class CdapIO {
+
+  public static <K, V> Read<K, V> read() {
+    return new AutoValue_CdapIO_Read.Builder<K, V>().build();
+  }
+
+  public static <K, V> Write<K, V> write() {
+    return new AutoValue_CdapIO_Write.Builder<K, V>().build();
+  }
+
+  /** A {@link PTransform} to read from CDAP source. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<K, V> extends PTransform<PBegin, PCollection<KV<K, V>>> {
+
+    abstract @Nullable PluginConfig getPluginConfig();
+
+    abstract @Nullable Plugin getCdapPlugin();
+
+    abstract @Nullable Class<K> getKeyClass();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract Builder<K, V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)
+    @AutoValue.Builder
+    abstract static class Builder<K, V> {
+
+      abstract Builder<K, V> setPluginConfig(PluginConfig config);
+
+      abstract Builder<K, V> setCdapPlugin(Plugin plugin);
+
+      abstract Builder<K, V> setKeyClass(Class<K> keyClass);
+
+      abstract Builder<K, V> setValueClass(Class<V> valueClass);
+
+      abstract Read<K, V> build();
+    }
+
+    public Read<K, V> withCdapPlugin(Plugin plugin) {
+      checkArgument(plugin != null, "Cdap plugin can not be null");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Read<K, V> withCdapPluginClass(Class<?> cdapPluginClass) {
+      checkArgument(cdapPluginClass != null, "Cdap plugin class can not be null");
+      Plugin plugin = MappingUtils.getPluginByClass(cdapPluginClass);
+      checkArgument(plugin != null, "Can not instantiate Plugin object from provided plugin class");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Read<K, V> withPluginConfig(PluginConfig pluginConfig) {
+      checkArgument(pluginConfig != null, "Plugin config can not be null");
+      return toBuilder().setPluginConfig(pluginConfig).build();
+    }
+
+    public Read<K, V> withKeyClass(Class<K> keyClass) {
+      checkArgument(keyClass != null, "Key class can not be null");
+      return toBuilder().setKeyClass(keyClass).build();
+    }
+
+    public Read<K, V> withValueClass(Class<V> valueClass) {
+      checkArgument(valueClass != null, "Value class can not be null");
+      return toBuilder().setValueClass(valueClass).build();
+    }
+
+    @Override
+    public PCollection<KV<K, V>> expand(PBegin input) {
+      validateTransform();
+
+      getCdapPlugin()
+          .withConfig(getPluginConfig())
+          .withHadoopConfiguration(getKeyClass(), getValueClass())
+          .prepareRun();
+
+      if (getCdapPlugin().isUnbounded()) {
+        // TODO: implement SparkReceiverIO.<~>read()
+        return null;
+      } else {
+        Configuration hConf = getCdapPlugin().getHadoopConfiguration();
+        HadoopFormatIO.Read<K, V> readFromHadoop =
+            HadoopFormatIO.<K, V>read().withConfiguration(hConf);
+        return input.apply(readFromHadoop);
+      }
+    }
+
+    public void validateTransform() {
+      checkArgument(getCdapPlugin() != null, "withCdapPluginClass() is required");
+      checkArgument(getPluginConfig() != null, "withPluginConfig() is required");
+      checkArgument(getKeyClass() != null, "withKeyClass() is required");
+      checkArgument(getValueClass() != null, "withValueClass() is required");
+    }
+  }
+
+  /** A {@link PTransform} to read from CDAP source. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Write<K, V> extends PTransform<PCollection<KV<K, V>>, PDone> {
+
+    abstract @Nullable PluginConfig getPluginConfig();
+
+    abstract @Nullable Plugin getCdapPlugin();
+
+    abstract @Nullable Class<K> getKeyClass();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract @Nullable String getLocksDirPath();
+
+    abstract Builder<K, V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)
+    @AutoValue.Builder
+    abstract static class Builder<K, V> {
+
+      abstract Builder<K, V> setPluginConfig(PluginConfig config);
+
+      abstract Builder<K, V> setCdapPlugin(Plugin plugin);
+
+      abstract Builder<K, V> setKeyClass(Class<K> keyClass);
+
+      abstract Builder<K, V> setValueClass(Class<V> valueClass);
+
+      abstract Builder<K, V> setLocksDirPath(String path);
+
+      abstract Write<K, V> build();
+    }
+
+    public Write<K, V> withCdapPlugin(Plugin plugin) {
+      checkArgument(plugin != null, "Cdap plugin can not be null");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Write<K, V> withCdapPluginClass(Class<?> cdapPluginClass) {
+      checkArgument(cdapPluginClass != null, "Cdap plugin class can not be null");
+      Plugin plugin = MappingUtils.getPluginByClass(cdapPluginClass);
+      checkArgument(plugin != null, "Can not instantiate Plugin object from provided plugin class");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Write<K, V> withPluginConfig(PluginConfig pluginConfig) {
+      checkArgument(pluginConfig != null, "Plugin config can not be null");
+      return toBuilder().setPluginConfig(pluginConfig).build();
+    }
+
+    public Write<K, V> withKeyClass(Class<K> keyClass) {
+      checkArgument(keyClass != null, "Key class can not be null");
+      return toBuilder().setKeyClass(keyClass).build();
+    }
+
+    public Write<K, V> withLocksDirPath(String locksDirPath) {
+      checkArgument(locksDirPath != null, "Locks dir path can not be null");
+      return toBuilder().setLocksDirPath(locksDirPath).build();
+    }
+
+    public Write<K, V> withValueClass(Class<V> valueClass) {
+      checkArgument(valueClass != null, "Value class can not be null");
+      return toBuilder().setValueClass(valueClass).build();
+    }
+
+    @Override
+    public PDone expand(PCollection<KV<K, V>> input) {
+      getCdapPlugin()
+          .withConfig(getPluginConfig())
+          .withHadoopConfiguration(getKeyClass(), getValueClass())
+          .prepareRun();
+
+      if (getCdapPlugin().isUnbounded()) {
+        // TODO: implement SparkReceiverIO.<~>write()

Review Comment:
   same as above



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchSinkContextImpl.java:
##########
@@ -24,7 +24,10 @@
 public class BatchSinkContextImpl extends BatchContextImpl implements BatchSinkContext {
 
   @Override
-  public void addOutput(Output output) {}
+  public void addOutput(Output output) {

Review Comment:
   this probably should be setoutput if it can only be one element
   
   if it is addoutput, it should be some form of a collection of elements



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.io.cdap;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import io.cdap.cdap.api.plugin.PluginConfig;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.io.hadoop.format.HDFSSynchronization;
+import org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.hadoop.conf.Configuration;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * An unbounded/bounded sources and sinks from <a
+ * href="https://github.com/data-integrations">CDAP</a> plugins.
+ */
+@SuppressWarnings({
+  "argument.type.incompatible",
+  "return.type.incompatible",
+  "dereference.of.nullable",
+  "UnnecessaryParentheses"
+})
+public class CdapIO {
+
+  public static <K, V> Read<K, V> read() {
+    return new AutoValue_CdapIO_Read.Builder<K, V>().build();
+  }
+
+  public static <K, V> Write<K, V> write() {
+    return new AutoValue_CdapIO_Write.Builder<K, V>().build();
+  }
+
+  /** A {@link PTransform} to read from CDAP source. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<K, V> extends PTransform<PBegin, PCollection<KV<K, V>>> {
+
+    abstract @Nullable PluginConfig getPluginConfig();
+
+    abstract @Nullable Plugin getCdapPlugin();
+
+    abstract @Nullable Class<K> getKeyClass();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract Builder<K, V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)
+    @AutoValue.Builder
+    abstract static class Builder<K, V> {
+
+      abstract Builder<K, V> setPluginConfig(PluginConfig config);
+
+      abstract Builder<K, V> setCdapPlugin(Plugin plugin);
+
+      abstract Builder<K, V> setKeyClass(Class<K> keyClass);
+
+      abstract Builder<K, V> setValueClass(Class<V> valueClass);
+
+      abstract Read<K, V> build();
+    }
+
+    public Read<K, V> withCdapPlugin(Plugin plugin) {
+      checkArgument(plugin != null, "Cdap plugin can not be null");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Read<K, V> withCdapPluginClass(Class<?> cdapPluginClass) {
+      checkArgument(cdapPluginClass != null, "Cdap plugin class can not be null");
+      Plugin plugin = MappingUtils.getPluginByClass(cdapPluginClass);
+      checkArgument(plugin != null, "Can not instantiate Plugin object from provided plugin class");
+      return toBuilder().setCdapPlugin(plugin).build();
+    }
+
+    public Read<K, V> withPluginConfig(PluginConfig pluginConfig) {
+      checkArgument(pluginConfig != null, "Plugin config can not be null");
+      return toBuilder().setPluginConfig(pluginConfig).build();
+    }
+
+    public Read<K, V> withKeyClass(Class<K> keyClass) {
+      checkArgument(keyClass != null, "Key class can not be null");
+      return toBuilder().setKeyClass(keyClass).build();
+    }
+
+    public Read<K, V> withValueClass(Class<V> valueClass) {
+      checkArgument(valueClass != null, "Value class can not be null");
+      return toBuilder().setValueClass(valueClass).build();
+    }
+
+    @Override
+    public PCollection<KV<K, V>> expand(PBegin input) {
+      validateTransform();
+
+      getCdapPlugin()
+          .withConfig(getPluginConfig())
+          .withHadoopConfiguration(getKeyClass(), getValueClass())
+          .prepareRun();
+
+      if (getCdapPlugin().isUnbounded()) {
+        // TODO: implement SparkReceiverIO.<~>read()
+        return null;
+      } else {
+        Configuration hConf = getCdapPlugin().getHadoopConfiguration();
+        HadoopFormatIO.Read<K, V> readFromHadoop =
+            HadoopFormatIO.<K, V>read().withConfiguration(hConf);
+        return input.apply(readFromHadoop);
+      }
+    }
+
+    public void validateTransform() {
+      checkArgument(getCdapPlugin() != null, "withCdapPluginClass() is required");
+      checkArgument(getPluginConfig() != null, "withPluginConfig() is required");
+      checkArgument(getKeyClass() != null, "withKeyClass() is required");
+      checkArgument(getValueClass() != null, "withValueClass() is required");
+    }
+  }
+
+  /** A {@link PTransform} to read from CDAP source. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Write<K, V> extends PTransform<PCollection<KV<K, V>>, PDone> {
+
+    abstract @Nullable PluginConfig getPluginConfig();
+
+    abstract @Nullable Plugin getCdapPlugin();
+
+    abstract @Nullable Class<K> getKeyClass();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract @Nullable String getLocksDirPath();
+
+    abstract Builder<K, V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)
+    @AutoValue.Builder
+    abstract static class Builder<K, V> {
+
+      abstract Builder<K, V> setPluginConfig(PluginConfig config);
+
+      abstract Builder<K, V> setCdapPlugin(Plugin plugin);
+
+      abstract Builder<K, V> setKeyClass(Class<K> keyClass);
+
+      abstract Builder<K, V> setValueClass(Class<V> valueClass);
+
+      abstract Builder<K, V> setLocksDirPath(String path);
+
+      abstract Write<K, V> build();
+    }
+
+    public Write<K, V> withCdapPlugin(Plugin plugin) {
+      checkArgument(plugin != null, "Cdap plugin can not be null");

Review Comment:
   same as above



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java:
##########
@@ -84,16 +88,34 @@ public PluginConfig getPluginConfig() {
    * validating connection to the CDAP sink/source and performing initial tuning.
    */
   public void prepareRun() {
+    PluginConfig pluginConfig = getPluginConfig();
+    if (pluginConfig == null) {

Review Comment:
   we should use preconditions.checkNotNull here as well



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.io.cdap;
+
+import io.cdap.plugin.common.SourceInputFormatProvider;
+import io.cdap.plugin.hubspot.sink.batch.HubspotBatchSink;
+import io.cdap.plugin.hubspot.sink.batch.HubspotOutputFormat;
+import io.cdap.plugin.hubspot.source.batch.HubspotBatchSource;
+import io.cdap.plugin.hubspot.source.batch.HubspotInputFormat;
+import io.cdap.plugin.hubspot.source.batch.HubspotInputFormatProvider;
+import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource;
+import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceInputFormat;
+import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceInputFormatProvider;
+import io.cdap.plugin.servicenow.source.ServiceNowInputFormat;
+import io.cdap.plugin.servicenow.source.ServiceNowSource;
+import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSource;
+import io.cdap.plugin.zendesk.source.batch.ZendeskInputFormat;
+import io.cdap.plugin.zendesk.source.batch.ZendeskInputFormatProvider;
+import javax.annotation.Nullable;
+
+public class MappingUtils {
+
+  public static @Nullable Plugin getPluginByClass(Class<?> pluginClass) {
+    if (pluginClass.equals(SalesforceBatchSource.class)) {
+      return Plugin.create(
+          pluginClass, SalesforceInputFormat.class, SalesforceInputFormatProvider.class);
+    } else if (pluginClass.equals(HubspotBatchSource.class)) {
+      return Plugin.create(pluginClass, HubspotInputFormat.class, HubspotInputFormatProvider.class);
+    } else if (pluginClass.equals(ZendeskBatchSource.class)) {
+      return Plugin.create(pluginClass, ZendeskInputFormat.class, ZendeskInputFormatProvider.class);
+    } else if (pluginClass.equals(HubspotBatchSink.class)) {
+      return Plugin.create(pluginClass, HubspotOutputFormat.class, SourceInputFormatProvider.class);
+    } else if (pluginClass.equals(ServiceNowSource.class)) {
+      return Plugin.create(
+          pluginClass, ServiceNowInputFormat.class, SourceInputFormatProvider.class);
+    }
+    return null;

Review Comment:
   this should throw an exception, not return null



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.io.cdap;
+
+import io.cdap.plugin.common.SourceInputFormatProvider;
+import io.cdap.plugin.hubspot.sink.batch.HubspotBatchSink;
+import io.cdap.plugin.hubspot.sink.batch.HubspotOutputFormat;
+import io.cdap.plugin.hubspot.source.batch.HubspotBatchSource;
+import io.cdap.plugin.hubspot.source.batch.HubspotInputFormat;
+import io.cdap.plugin.hubspot.source.batch.HubspotInputFormatProvider;
+import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource;
+import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceInputFormat;
+import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceInputFormatProvider;
+import io.cdap.plugin.servicenow.source.ServiceNowInputFormat;
+import io.cdap.plugin.servicenow.source.ServiceNowSource;
+import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSource;
+import io.cdap.plugin.zendesk.source.batch.ZendeskInputFormat;
+import io.cdap.plugin.zendesk.source.batch.ZendeskInputFormatProvider;
+import javax.annotation.Nullable;
+
+public class MappingUtils {
+
+  public static @Nullable Plugin getPluginByClass(Class<?> pluginClass) {
+    if (pluginClass.equals(SalesforceBatchSource.class)) {

Review Comment:
   Should this be a static method, or is there a way we can enable a method to register new plugins?



##########
sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java:
##########
@@ -84,16 +88,34 @@ public PluginConfig getPluginConfig() {
    * validating connection to the CDAP sink/source and performing initial tuning.
    */
   public void prepareRun() {
+    PluginConfig pluginConfig = getPluginConfig();
+    if (pluginConfig == null) {
+      throw new IllegalArgumentException("PluginConfig should be not null!");
+    }
     if (cdapPluginObj == null) {
-      InstantiatorFactory instantiatorFactory = new InstantiatorFactory(false);
-      cdapPluginObj =
-          (SubmitterLifecycle) instantiatorFactory.get(TypeToken.of(getPluginClass())).create();
+      for (Constructor<?> constructor : getPluginClass().getDeclaredConstructors()) {

Review Comment:
   this is odd to me. It seems like we shouldn't be grabbing all the constructors, but just the one that can be constructed with the plugin config.
   
   If we have multiple constructors here, one will certainly cause the exception block to be hit, even if the other constructor is valid



-- 
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: github-unsubscribe@beam.apache.org

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