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 2020/12/30 17:04:34 UTC

[GitHub] [beam] TheNeuralBit commented on a change in pull request #13622: [BEAM-11530] Annotated setter parameters handled wrong in schema creation

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



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
##########
@@ -176,39 +177,38 @@ public static FieldValueTypeInformation forGetter(Method method) {
   }
 
   private static boolean hasNullableAnnotation(Field field) {
-    for (Annotation annotation : field.getAnnotations()) {
-      if (isNullableAnnotation(annotation)) {
-        return true;
-      }
-    }
-
-    for (Annotation annotation : field.getAnnotatedType().getAnnotations()) {
-      if (isNullableAnnotation(annotation)) {
-        return true;
-      }
-    }
+    Stream<Annotation> annotations =
+        Stream.concat(
+            Stream.of(field.getAnnotations()),
+            Stream.of(field.getAnnotatedType().getAnnotations()));
 
-    return false;
+    return annotations.anyMatch(FieldValueTypeInformation::isNullableAnnotation);
   }
 
   /**
    * If the method or its return type are annotated with any variant of Nullable, then the schema
    * field is nullable.
    */
   private static boolean hasNullableReturnType(Method method) {
-    for (Annotation annotation : method.getAnnotations()) {
-      if (isNullableAnnotation(annotation)) {
-        return true;
-      }
-    }
+    Stream<Annotation> annotations =
+        Stream.concat(
+            Stream.of(method.getAnnotations()),
+            Stream.of(method.getAnnotatedReturnType().getAnnotations()));
 
-    for (Annotation annotation : method.getAnnotatedReturnType().getAnnotations()) {
-      if (isNullableAnnotation(annotation)) {
-        return true;
-      }
+    return annotations.anyMatch(FieldValueTypeInformation::isNullableAnnotation);
+  }
+
+  private static boolean hasSingleNullableParameter(Method method) {
+    if (method.getParameterCount() != 1) {
+      throw new RuntimeException("Setter methods should take a single argument.");

Review comment:
       nit: Let's add the method name to this exception so it's more actionable
   ```suggestion
         throw new RuntimeException("Setter methods should take a single argument " + method.getName());
   ```

##########
File path: sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java
##########
@@ -216,6 +219,39 @@ public void testNullableFromRow() throws NoSuchSchemaException {
     assertNull(bean.getStringBuilder());
   }
 
+  /**
+   * [BEAM-11530] Java distinguishes between parameter annotations and type annotations. Therefore
+   * annotations declared without {@link java.lang.annotation.ElementType#TYPE_USE} can't be
+   * accessed through {@link Executable#getAnnotatedParameterTypes()}. Some {@code @Nullable}
+   * annotations like {@link org.apache.avro.reflect.Nullable} do not declare {@link
+   * java.lang.annotation.ElementType#TYPE_USE} which makes them parameter annotations once placed
+   * in front of a parameter.
+   *
+   * @see <a
+   *     href="https://stackoverflow.com/a/37587590/5896429">https://stackoverflow.com/a/37587590/5896429</a>
+   * @see <a
+   *     href="https://github.com/apache/beam/pull/13622">https://github.com/apache/beam/pull/13622</a>
+   */

Review comment:
       Thank you this works :)
   
   nit: we don't need the link to the PR, it can be found through `git blame` or the jira
   ```suggestion
      */
   ```




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

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