You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by bh...@apache.org on 2020/12/30 20:56:08 UTC

[beam] branch master updated: [BEAM-11530] Consider all kinds of annotatios on schema creation (#13622)

This is an automated email from the ASF dual-hosted git repository.

bhulette pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new fe0dfa1  [BEAM-11530] Consider all kinds of annotatios on schema creation (#13622)
fe0dfa1 is described below

commit fe0dfa1b34b06290adfa074ecda494a4f1cf7c56
Author: Filip Krakowski <kr...@hhu.de>
AuthorDate: Wed Dec 30 21:55:24 2020 +0100

    [BEAM-11530] Consider all kinds of annotatios on schema creation (#13622)
---
 CHANGES.md                                         |  1 +
 .../sdk/schemas/FieldValueTypeInformation.java     | 53 ++++++++++------------
 .../beam/sdk/schemas/JavaBeanSchemaTest.java       | 34 ++++++++++++++
 .../beam/sdk/schemas/utils/TestJavaBeans.java      | 17 +++++++
 4 files changed, 77 insertions(+), 28 deletions(-)

diff --git a/CHANGES.md b/CHANGES.md
index 263c4b1..5e82be6 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -72,6 +72,7 @@
 
 ## Known Issues
 
+* Fixed annotation handling for setter parameters in schema creation (Java) ([BEAM-11530](https://issues.apache.org/jira/browse/BEAM-11530)).
 * Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
 
 # [2.27.0] - Cut, Unreleased
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
index e512a05..0220970 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
@@ -27,6 +27,7 @@ import java.lang.reflect.Method;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
+import java.util.stream.Stream;
 import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat;
 import org.apache.beam.sdk.schemas.annotations.SchemaFieldName;
 import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
@@ -176,19 +177,12 @@ public abstract class FieldValueTypeInformation implements Serializable {
   }
 
   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);
   }
 
   /**
@@ -196,19 +190,26 @@ public abstract class FieldValueTypeInformation implements Serializable {
    * 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 " + method.getName());
     }
 
-    return false;
+    Stream<Annotation> annotations =
+        Stream.concat(
+            Arrays.stream(method.getAnnotatedParameterTypes()[0].getAnnotations()),
+            Arrays.stream(method.getParameterAnnotations()[0]));
+
+    return annotations.anyMatch(FieldValueTypeInformation::isNullableAnnotation);
   }
 
   /** Try to accept any Nullable annotation. */
@@ -227,13 +228,9 @@ public abstract class FieldValueTypeInformation implements Serializable {
     } else {
       throw new RuntimeException("Setter has wrong prefix " + method.getName());
     }
-    if (method.getParameterCount() != 1) {
-      throw new RuntimeException("Setter methods should take a single argument.");
-    }
+
     TypeDescriptor type = TypeDescriptor.of(method.getGenericParameterTypes()[0]);
-    boolean nullable =
-        Arrays.stream(method.getParameters()[0].getAnnotatedType().getAnnotations())
-            .anyMatch(annotation -> isNullableAnnotation(annotation));
+    boolean nullable = hasSingleNullableParameter(method);
     return new AutoValue_FieldValueTypeInformation.Builder()
         .setName(name)
         .setNullable(nullable)
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java
index 9833ace..9b2d23c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java
@@ -26,6 +26,7 @@ import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.NESTED_ARRAYS_BEAM
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.NESTED_ARRAY_BEAN_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.NESTED_BEAN_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.NESTED_MAP_BEAN_SCHEMA;
+import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.PARAMETER_NULLABLE_BEAN_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.PRIMITIVE_ARRAY_BEAN_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.RENAMED_FIELDS_AND_SETTERS_BEAM_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.SIMPLE_BEAN_SCHEMA;
@@ -38,6 +39,7 @@ import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 
+import java.lang.reflect.Executable;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
@@ -56,6 +58,7 @@ import org.apache.beam.sdk.schemas.utils.TestJavaBeans.NestedArrayBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.NestedArraysBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.NestedBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.NestedMapBean;
+import org.apache.beam.sdk.schemas.utils.TestJavaBeans.ParameterNullableBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.PrimitiveArrayBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.SimpleBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.SimpleBeanWithAnnotations;
@@ -216,6 +219,37 @@ public class JavaBeanSchemaTest {
     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>
+   */
+  @Test
+  public void testParameterNullableToRow() throws NoSuchSchemaException {
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    ParameterNullableBean bean = new ParameterNullableBean();
+    Row row = registry.getToRowFunction(ParameterNullableBean.class).apply(bean);
+
+    assertEquals(1, row.getFieldCount());
+    assertNull(row.getInt64("value"));
+  }
+
+  @Test
+  public void testParameterNullableFromRow() throws NoSuchSchemaException {
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    Row row = Row.nullRow(PARAMETER_NULLABLE_BEAN_SCHEMA);
+
+    ParameterNullableBean bean =
+        registry.getFromRowFunction(ParameterNullableBean.class).apply(row);
+    assertNull(bean.getValue());
+  }
+
   @Test
   public void testToRowSerializable() throws NoSuchSchemaException {
     SchemaRegistry registry = SchemaRegistry.createDefault();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java
index 0911873..bd25428 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java
@@ -1330,4 +1330,21 @@ public class TestJavaBeans {
           .addInt32Field("age_in_years")
           .addBooleanField("KnowsJavascript")
           .build();
+
+  @DefaultSchema(JavaBeanSchema.class)
+  public static class ParameterNullableBean {
+
+    @org.apache.avro.reflect.Nullable private Float value;
+
+    public @org.apache.avro.reflect.Nullable Float getValue() {
+      return value;
+    }
+
+    public void setValue(@org.apache.avro.reflect.Nullable Float value) {
+      this.value = value;
+    }
+  }
+
+  public static final Schema PARAMETER_NULLABLE_BEAN_SCHEMA =
+      Schema.builder().addNullableField("value", FieldType.INT64).build();
 }