[ 
https://issues.apache.org/jira/browse/BEAM-4454?focusedWorklogId=171389&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-171389
 ]

ASF GitHub Bot logged work on BEAM-4454:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/Dec/18 16:55
            Start Date: 02/Dec/18 16:55
    Worklog Time Spent: 10m 
      Work Description: kanterov commented on a change in pull request #7181: 
[BEAM-4454] Add more AVRO utilities to convert between Beam and Avro.
URL: https://github.com/apache/beam/pull/7181#discussion_r238103753
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
 ##########
 @@ -20,87 +20,210 @@
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.avro.Schema.Type;
 import org.apache.avro.generic.GenericEnumSymbol;
 import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.avro.reflect.ReflectData;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.values.Row;
 
 /** Utils to convert AVRO records to Beam rows. */
 @Experimental(Experimental.Kind.SCHEMAS)
 public class AvroUtils {
+  // Unwrap an AVRO schema into the base type an whether it is nullable.
+  static class TypeWithNullability {
+    public final org.apache.avro.Schema type;
+    public final boolean nullable;
+
+    TypeWithNullability(org.apache.avro.Schema avroSchema) {
+      if (avroSchema.getType() == org.apache.avro.Schema.Type.UNION) {
+        List<org.apache.avro.Schema> types = avroSchema.getTypes();
+
+        // optional fields in AVRO have form of:
+        // {"name": "foo", "type": ["null", "something"]}
+
+        // don't need recursion because nested unions aren't supported in AVRO
+        List<org.apache.avro.Schema> nonNullTypes =
+            types
+                .stream()
+                .filter(x -> x.getType() != org.apache.avro.Schema.Type.NULL)
+                .collect(Collectors.toList());
+
+        if (nonNullTypes.size() == types.size() || nonNullTypes.isEmpty()) {
+          // union without `null` or all 'null' union, keep as is.
+          type = avroSchema;
+          nullable = false;
+        } else if (nonNullTypes.size() > 1) {
+          type = org.apache.avro.Schema.createUnion(nonNullTypes);
+          nullable = true;
+        } else {
+          // One non-null type.
+          type = nonNullTypes.get(0);
+          nullable = true;
+        }
+      } else {
+        type = avroSchema;
+        nullable = false;
+      }
+    }
+  }
+
   private AvroUtils() {}
 
   /**
    * Converts AVRO schema to Beam row schema.
    *
    * @param schema schema of type RECORD
    */
-  public static Schema toSchema(@Nonnull org.apache.avro.Schema schema) {
+  public static Schema toBeamSchema(org.apache.avro.Schema schema) {
 
 Review comment:
   nit: we had a discussion with @akedin 
https://github.com/apache/beam/pull/6118#discussion_r206980368 that our goal is 
to drop `Beam` prefix gradually (e.g. `BeamRecord` -> `Row`). I don't mind 
having a prefix or not, in fact, I initially used it myself, but I think we 
should be consistent.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 171389)
    Time Spent: 1h 10m  (was: 1h)

> Provide automatic schema registration for AVROs
> -----------------------------------------------
>
>                 Key: BEAM-4454
>                 URL: https://issues.apache.org/jira/browse/BEAM-4454
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-java-core
>            Reporter: Reuven Lax
>            Assignee: Reuven Lax
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Need to make sure this is a compatible change



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to