funcheetah commented on code in PR #4242:
URL: https://github.com/apache/iceberg/pull/4242#discussion_r854343442


##########
core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java:
##########
@@ -154,6 +154,27 @@ public static boolean isOptionSchema(Schema schema) {
     return false;
   }
 
+  /**
+   * This method decides whether a schema is of type union and is complex 
union and is optional
+   *
+   * Complex union: the number of options in union not equals to 2
+   * Optional: null is present in union
+   *
+   * @param schema input schema
+   * @return true if schema is complex union and it is optional
+   */
+  public static boolean isOptionalComplexUnion(Schema schema) {
+    if (schema.getType() == UNION && schema.getTypes().size() != 2) {

Review Comment:
   Thanks for catching this. Non-nullable union of single type will be handled 
in follow-up PR. Basically, ```[type]``` will be read as ```type``` instead of 
```struct<tag, field0>```.



##########
core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java:
##########
@@ -52,8 +52,21 @@
       case UNION:
         List<Schema> types = schema.getTypes();
         List<T> options = Lists.newArrayListWithExpectedSize(types.size());
-        for (Schema type : types) {
-          options.add(visit(type, visitor));
+        if (AvroSchemaUtil.isOptionSchema(schema)) {
+          for (Schema type : types) {
+            options.add(visit(type, visitor));
+          }
+        } else {
+          // complex union case

Review Comment:
   The union to struct conversion and assigning filed names need to happen 
while traversing the schema tree. We put this logic in a separate branch other 
than optional union. In addition, the idea is similar to how we handle 
```LogicalMap``` as in the code:
   ```
         case ARRAY:
           if (schema.getLogicalType() instanceof LogicalMap) {
             return visitor.array(schema, visit(schema.getElementType(), 
visitor));
           } else {
             return visitor.array(schema, visitWithName("element", 
schema.getElementType(), visitor));
           }
   ```
   
   This branch of code which handles cases other than optional union ```[null, 
type]``` is needed. If we remove it, tests will fail. Curious which tests are 
you running? I can see the tests failure in both 
```TestUnionSchemaConversions``` and ```TestSparkAvroUnions```.
   
   
   



##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroUnions.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * Licensed 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.iceberg.spark.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.AvroIterable;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestSparkAvroUnions {

Review Comment:
   Thanks for the suggestions. Projection of columns inside a complex union is 
not supported yet and will be handled in followup PR. Do you think it is okay 
to add tests for columns projection later when it is supported?



##########
core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java:
##########
@@ -79,11 +79,25 @@
   private static <T> T visitUnion(Type type, Schema union, 
AvroSchemaWithTypeVisitor<T> visitor) {
     List<Schema> types = union.getTypes();
     List<T> options = Lists.newArrayListWithExpectedSize(types.size());
-    for (Schema branch : types) {
-      if (branch.getType() == Schema.Type.NULL) {
-        options.add(visit((Type) null, branch, visitor));
-      } else {
-        options.add(visit(type, branch, visitor));
+
+    // simple union case
+    if (AvroSchemaUtil.isOptionSchema(union)) {
+      for (Schema branch : types) {
+        if (branch.getType() == Schema.Type.NULL) {
+          options.add(visit((Type) null, branch, visitor));
+        } else {
+          options.add(visit(type, branch, visitor));
+        }
+      }
+    } else { // complex union case
+      int index = 1;
+      for (Schema branch : types) {
+        if (branch.getType() == Schema.Type.NULL) {
+          options.add(visit((Type) null, branch, visitor));
+        } else {
+          options.add(visit(type.asStructType().fields().get(index).type(), 
branch, visitor));

Review Comment:
   Replied to the comment to ```AvroSchemaVisitor```, please take a look. 
Thanks.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to