rdblue commented on code in PR #11988:
URL: https://github.com/apache/iceberg/pull/11988#discussion_r1919506979


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java:
##########
@@ -244,6 +286,285 @@ public void testTimestampWithoutZone() throws IOException 
{
         });
   }
 
+  @Test
+  public void testMissingRequiredWithoutDefault() {
+    Assumptions.assumeThat(supportsDefaultValues()).isTrue();
+
+    Schema writeSchema = new Schema(required(1, "id", Types.LongType.get()));
+
+    Schema expectedSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.required("missing_str")
+                .withId(6)
+                .ofType(Types.StringType.get())
+                .withDoc("Missing required field with no default")
+                .build());
+
+    assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema))
+        .has(
+            new Condition<>(
+                t ->
+                    IllegalArgumentException.class.isInstance(t)
+                        || 
IllegalArgumentException.class.isInstance(t.getCause()),
+                "Expecting a throwable or cause that is an instance of 
IllegalArgumentException"))
+        .hasMessageContaining("Missing required field: missing_str");
+  }
+
+  @Test
+  public void testDefaultValues() throws IOException {
+    Assumptions.assumeThat(supportsDefaultValues()).isTrue();
+
+    Schema writeSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .withDoc("Should not produce default value")
+                .build());
+
+    Schema expectedSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .build(),
+            Types.NestedField.required("missing_str")
+                .withId(6)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("orange")
+                .build(),
+            Types.NestedField.optional("missing_int")
+                .withId(7)
+                .ofType(Types.IntegerType.get())
+                .withInitialDefault(34)
+                .build());
+
+    writeAndValidate(writeSchema, expectedSchema);
+  }
+
+  @Test
+  public void testNullDefaultValue() throws IOException {
+    Assumptions.assumeThat(supportsDefaultValues()).isTrue();
+
+    Schema writeSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .withDoc("Should not produce default value")
+                .build());
+
+    Schema expectedSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .build(),
+            Types.NestedField.optional("missing_date")
+                .withId(3)
+                .ofType(Types.DateType.get())
+                .build());
+
+    writeAndValidate(writeSchema, expectedSchema);
+  }
+
+  @Test
+  public void testNestedDefaultValue() throws IOException {
+    Assumptions.assumeThat(supportsDefaultValues()).isTrue();
+    Assumptions.assumeThat(supportsNestedTypes()).isTrue();
+
+    Schema writeSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .withDoc("Should not produce default value")
+                .build(),
+            Types.NestedField.optional("nested")
+                .withId(3)
+                .ofType(Types.StructType.of(required(4, "inner", 
Types.StringType.get())))
+                .withDoc("Used to test nested field defaults")
+                .build());
+
+    Schema expectedSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .build(),
+            Types.NestedField.optional("nested")
+                .withId(3)
+                .ofType(
+                    Types.StructType.of(
+                        required(4, "inner", Types.StringType.get()),
+                        Types.NestedField.optional("missing_inner_float")
+                            .withId(5)
+                            .ofType(Types.FloatType.get())
+                            .withInitialDefault(-0.0F)
+                            .build()))
+                .withDoc("Used to test nested field defaults")
+                .build());
+
+    writeAndValidate(writeSchema, expectedSchema);
+  }
+
+  @Test
+  public void testMapNestedDefaultValue() throws IOException {
+    Assumptions.assumeThat(supportsDefaultValues()).isTrue();
+    Assumptions.assumeThat(supportsNestedTypes()).isTrue();
+
+    Schema writeSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .withDoc("Should not produce default value")
+                .build(),
+            Types.NestedField.optional("nested_map")
+                .withId(3)
+                .ofType(
+                    Types.MapType.ofOptional(
+                        4,
+                        5,
+                        Types.StringType.get(),
+                        Types.StructType.of(required(6, "value_str", 
Types.StringType.get()))))
+                .withDoc("Used to test nested map value field defaults")
+                .build());
+
+    Schema expectedSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .build(),
+            Types.NestedField.optional("nested_map")
+                .withId(3)
+                .ofType(
+                    Types.MapType.ofOptional(
+                        4,
+                        5,
+                        Types.StringType.get(),
+                        Types.StructType.of(
+                            required(6, "value_str", Types.StringType.get()),
+                            Types.NestedField.optional("value_int")
+                                .withId(7)
+                                .ofType(Types.IntegerType.get())
+                                .withInitialDefault(34)
+                                .build())))
+                .withDoc("Used to test nested field defaults")
+                .build());
+
+    writeAndValidate(writeSchema, expectedSchema);
+  }
+
+  @Test
+  public void testListNestedDefaultValue() throws IOException {
+    Assumptions.assumeThat(supportsDefaultValues()).isTrue();
+    Assumptions.assumeThat(supportsNestedTypes()).isTrue();
+
+    Schema writeSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .withDoc("Should not produce default value")
+                .build(),
+            Types.NestedField.optional("nested_list")
+                .withId(3)
+                .ofType(
+                    Types.ListType.ofOptional(
+                        4, Types.StructType.of(required(5, "element_str", 
Types.StringType.get()))))
+                .withDoc("Used to test nested field defaults")
+                .build());
+
+    Schema expectedSchema =
+        new Schema(
+            required(1, "id", Types.LongType.get()),
+            Types.NestedField.optional("data")
+                .withId(2)
+                .ofType(Types.StringType.get())
+                .withInitialDefault("wrong!")
+                .build(),
+            Types.NestedField.optional("nested_list")
+                .withId(3)
+                .ofType(
+                    Types.ListType.ofOptional(
+                        4,
+                        Types.StructType.of(
+                            required(5, "element_str", Types.StringType.get()),
+                            Types.NestedField.optional("element_int")
+                                .withId(7)
+                                .ofType(Types.IntegerType.get())
+                                .withInitialDefault(34)
+                                .build())))
+                .withDoc("Used to test nested field defaults")
+                .build());
+
+    writeAndValidate(writeSchema, expectedSchema);
+  }
+
+  private static Stream<Arguments> primitiveTypesAndDefaults() {
+    return Stream.of(
+        Arguments.of(Types.BooleanType.get(), false),
+        Arguments.of(Types.IntegerType.get(), 34),
+        Arguments.of(Types.LongType.get(), 4900000000L),
+        Arguments.of(Types.FloatType.get(), 12.21F),
+        Arguments.of(Types.DoubleType.get(), -0.0D),
+        Arguments.of(Types.DateType.get(), 
DateTimeUtil.isoDateToDays("2024-12-17")),
+        // Arguments.of(Types.TimeType.get(), 
DateTimeUtil.isoTimeToMicros("23:59:59.999999")),
+        Arguments.of(
+            Types.TimestampType.withZone(),
+            
DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")),
+        // Arguments.of(
+        //     Types.TimestampType.withoutZone(),
+        //     
DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")),

Review Comment:
   Spark 3.3 doesn't support TimestampNTZ without a flag, so this 3.3 backport 
doesn't remove `withSQLConf` below or the `testTimestampWithoutZone` case. It 
also doesn't use `TimestampType.withoutZone()` in default tests or in tests 
that use `SUPPORTED_PRIMITIVES`.



-- 
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: issues-unsubscr...@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to