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/07 13:01:15 UTC

[GitHub] [beam] aromanenko-dev commented on a diff in pull request #17111: [BEAM-14101] [CdapIO] Add ReceiverBuilder for SparkReceiverIO

aromanenko-dev commented on code in PR #17111:
URL: https://github.com/apache/beam/pull/17111#discussion_r891168171


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/package-info.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/** Transforms for reading and writing from streaming CDAP plugins. */
+@Experimental(Kind.SOURCE_SINK)
+package org.apache.beam.sdk.io.sparkreceiver;

Review Comment:
   I don't think we need a dedicated IO package for this since this builder, IIUC, is mostly a part of CdapIO and doesn't provide any read/write IO API for users.



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReceiverBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.sparkreceiver;
+
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import javax.annotation.Nullable;
+import org.apache.commons.lang.ClassUtils;
+import org.apache.spark.streaming.receiver.Receiver;
+
+/**
+ * Class for building an instance for {@link Receiver} that uses Apache Beam mechanisms instead of
+ * Spark environment.
+ */
+@SuppressWarnings({
+  "unchecked",
+  "argument.type.incompatible",
+  "return.type.incompatible",
+  "dereference.of.nullable"
+})
+public class ReceiverBuilder<X, T extends Receiver<X>> implements Serializable {
+
+  private final Class<T> sparkReceiverClass;
+  private @Nullable Object[] constructorArgs;
+
+  public ReceiverBuilder(Class<T> sparkReceiverClass) {
+    this.sparkReceiverClass = sparkReceiverClass;
+  }
+
+  /** Method for specifying constructor arguments for corresponding {@link #sparkReceiverClass}. */
+  public ReceiverBuilder<X, T> withConstructorArgs(Object... args) {
+    this.constructorArgs = args;
+    return this;
+  }
+
+  /**
+   * @return Proxy for given {@param receiver} that doesn't use Spark environment and uses Apache
+   *     Beam mechanisms instead.
+   */
+  public T build()
+      throws InvocationTargetException, InstantiationException, IllegalAccessException {
+
+    if (constructorArgs == null) {

Review Comment:
   nit: use `checkState()`



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReceiverBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.sparkreceiver;
+
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import javax.annotation.Nullable;
+import org.apache.commons.lang.ClassUtils;
+import org.apache.spark.streaming.receiver.Receiver;
+
+/**
+ * Class for building an instance for {@link Receiver} that uses Apache Beam mechanisms instead of
+ * Spark environment.
+ */
+@SuppressWarnings({
+  "unchecked",
+  "argument.type.incompatible",
+  "return.type.incompatible",
+  "dereference.of.nullable"
+})
+public class ReceiverBuilder<X, T extends Receiver<X>> implements Serializable {
+
+  private final Class<T> sparkReceiverClass;
+  private @Nullable Object[] constructorArgs;
+
+  public ReceiverBuilder(Class<T> sparkReceiverClass) {
+    this.sparkReceiverClass = sparkReceiverClass;
+  }
+
+  /** Method for specifying constructor arguments for corresponding {@link #sparkReceiverClass}. */
+  public ReceiverBuilder<X, T> withConstructorArgs(Object... args) {
+    this.constructorArgs = args;
+    return this;
+  }
+
+  /**
+   * @return Proxy for given {@param receiver} that doesn't use Spark environment and uses Apache
+   *     Beam mechanisms instead.
+   */
+  public T build()
+      throws InvocationTargetException, InstantiationException, IllegalAccessException {
+
+    if (constructorArgs == null) {
+      throw new IllegalStateException(
+          "It is not possible to build a Receiver proxy without setting the obligatory parameters.");
+    }
+    Constructor<?> currentConstructor = null;
+    for (Constructor<?> constructor : sparkReceiverClass.getDeclaredConstructors()) {
+      Class<?>[] paramTypes = constructor.getParameterTypes();
+      if (paramTypes.length != constructorArgs.length) {
+        continue;
+      }
+      boolean matches = true;
+      for (int i = 0; i < constructorArgs.length; i++) {
+        Object arg = constructorArgs[i];
+        if (arg == null) {
+          throw new IllegalArgumentException("All args must be not null!");
+        }
+        Class<?> currArgClass = paramTypes[i];
+        if (currArgClass.isPrimitive()) {
+          currArgClass = ClassUtils.primitiveToWrapper(currArgClass);
+        }
+        if (!currArgClass.equals(arg.getClass())) {
+          matches = false;
+          break;
+        }
+      }
+      if (matches) {
+        currentConstructor = constructor;
+      }
+    }
+    if (currentConstructor == null) {

Review Comment:
   nit: use `checkState()`



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReceiverBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.sparkreceiver;
+
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import javax.annotation.Nullable;
+import org.apache.commons.lang.ClassUtils;
+import org.apache.spark.streaming.receiver.Receiver;
+
+/**
+ * Class for building an instance for {@link Receiver} that uses Apache Beam mechanisms instead of
+ * Spark environment.
+ */
+@SuppressWarnings({

Review Comment:
   Could these warnings below be fixed instead of suppress?



##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -549,6 +549,7 @@ class BeamModulePlugin implements Plugin<Project> {
         commons_compress                            : "org.apache.commons:commons-compress:1.21",
         commons_csv                                 : "org.apache.commons:commons-csv:1.8",
         commons_io                                  : "commons-io:commons-io:2.6",
+        commons_lang2_6                             : "commons-lang:commons-lang:2.6",

Review Comment:
   Why this deps is needed?



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReceiverBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.sparkreceiver;
+
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import javax.annotation.Nullable;
+import org.apache.commons.lang.ClassUtils;
+import org.apache.spark.streaming.receiver.Receiver;
+
+/**
+ * Class for building an instance for {@link Receiver} that uses Apache Beam mechanisms instead of
+ * Spark environment.
+ */
+@SuppressWarnings({
+  "unchecked",
+  "argument.type.incompatible",
+  "return.type.incompatible",
+  "dereference.of.nullable"
+})
+public class ReceiverBuilder<X, T extends Receiver<X>> implements Serializable {
+
+  private final Class<T> sparkReceiverClass;
+  private @Nullable Object[] constructorArgs;
+
+  public ReceiverBuilder(Class<T> sparkReceiverClass) {
+    this.sparkReceiverClass = sparkReceiverClass;
+  }
+
+  /** Method for specifying constructor arguments for corresponding {@link #sparkReceiverClass}. */
+  public ReceiverBuilder<X, T> withConstructorArgs(Object... args) {
+    this.constructorArgs = args;
+    return this;
+  }
+
+  /**
+   * @return Proxy for given {@param receiver} that doesn't use Spark environment and uses Apache
+   *     Beam mechanisms instead.
+   */
+  public T build()
+      throws InvocationTargetException, InstantiationException, IllegalAccessException {
+
+    if (constructorArgs == null) {
+      throw new IllegalStateException(
+          "It is not possible to build a Receiver proxy without setting the obligatory parameters.");
+    }
+    Constructor<?> currentConstructor = null;
+    for (Constructor<?> constructor : sparkReceiverClass.getDeclaredConstructors()) {
+      Class<?>[] paramTypes = constructor.getParameterTypes();
+      if (paramTypes.length != constructorArgs.length) {
+        continue;
+      }
+      boolean matches = true;
+      for (int i = 0; i < constructorArgs.length; i++) {
+        Object arg = constructorArgs[i];
+        if (arg == null) {

Review Comment:
   nit: use `checkArgument()`



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