Makes perfect sense, ensuring the safety of schema evolution (no breaking
changes wrt loading data) is the rationale I was looking for, thank you.

On Wed, Jan 30, 2019, 19:25 Ryan Blue <rb...@netflix.com wrote:

> The built-in schema update doesn't allow you to add required fields
> because that would break schema evolution.
>
> Iceberg guarantees that the current schema can read all existing data
> files in a table, as long as it was evolved using the rules enforced by
> SchemaUpdate. One of those rules is that new columns must be optional
> because existing files could be written without them.
>
> If you can guarantee that the data is present in all files currently in
> the table, you can edit the schema to make the change. This isn't in the
> API because it requires a lot of knowledge and judgement about when it is
> safe.
>
> We could add an API for making unsafe changes to make that easier for
> administrators.
>
> On Wed, Jan 30, 2019 at 1:43 AM filip <filip....@gmail.com> wrote:
>
>> Thank you for the details Ryan but I think I was quite vague on the
>> initial question so please let me try rephrasing the question by adding
>> more context.
>> Say after creating an Iceberg table with a particular schema, for which
>> you can define top-level REQUIRED or OPTIONAL primitives, how can one
>> evolve the schema with yet more REQUIRED top-level primitives?
>>
>> I've worked out a tiny small test off because it seems that using
>> addColumn(String name, Type type) [1] has all top-level fields added to the
>> schema as optional. Any way I could add/ update as required fields instead?
>> I couldn't see an explicit solution in the add top-level field API that
>> accommodates the required/ optional aspect hence my question whether there
>> was an explicit API design choice of having top-level fields implicitly
>> added as optional.
>>
>> [1]
>> https://github.com/Netflix/iceberg/blob/master/core/src/main/java/com/netflix/iceberg/SchemaUpdate.java#L64
>>
>> This test fails because all fields are added as optional not required.
>>
>>   @Test
>>   public void testAddRequiredTopLevelPrimitives() {
>>     Schema schema = new Schema(
>>         required(1, "id", Types.IntegerType.get()));
>>
>>     Schema result = new SchemaUpdate(schema, 1)
>>         .addColumn("binary", Types.BinaryType.get())
>>         .addColumn("boolean", Types.BooleanType.get())
>>         .addColumn("date", Types.DateType.get())
>>         .addColumn("decimal", Types.DecimalType.of(38, 5))
>>         .addColumn("double", Types.DoubleType.get())
>>         .addColumn("fixed", Types.FixedType.ofLength(12))
>>         .addColumn("float", Types.FloatType.get())
>>         .addColumn("long", Types.LongType.get())
>>         .addColumn("string", Types.StringType.get())
>>         .addColumn("time", Types.TimeType.get())
>>         .addColumn("timestampz", Types.TimestampType.withoutZone())
>>         .addColumn("timestamp", Types.TimestampType.withZone())
>>         .addColumn("uuid", Types.UUIDType.get())
>>         .apply();
>>
>>     Schema expected = new Schema(
>>         required(1, "id", Types.IntegerType.get()),
>>         required(2, "binary", Types.BinaryType.get()),
>>         required(3, "boolean", Types.BooleanType.get()),
>>         required(4, "date", Types.DateType.get()),
>>         required(5, "decimal", Types.DecimalType.of(38, 5)),
>>         required(6, "double", Types.DoubleType.get()),
>>         required(8, "fixed", Types.FixedType.ofLength(12)),
>>         required(9, "float", Types.FloatType.get()),
>>         required(10, "long", Types.LongType.get()),
>>         required(11, "string", Types.StringType.get()),
>>         required(12, "time", Types.TimeType.get()),
>>         required(13, "timestampz", Types.TimestampType.withoutZone()),
>>         required(14, "timestamp", Types.TimestampType.withZone()),
>>         required(15, "uuid", Types.UUIDType.get())
>>     );
>>
>>     Assert.assertEquals("Should add required top level primitives and
>> assign column IDs",
>>         expected.asStruct(), result.asStruct());
>>   }
>>
>> On Tue, Jan 29, 2019 at 9:43 PM Ryan Blue <rb...@netflix.com.invalid>
>> wrote:
>>
>>> Hi Filip,
>>>
>>> Iceberg can add fields to any struct. You can see the test case here:
>>>
>>> https://github.com/apache/incubator-iceberg/blob/master/core/src/test/java/com/netflix/iceberg/TestSchemaUpdate.java#L264-L271
>>>
>>> rb
>>>
>>> On Tue, Jan 29, 2019 at 11:34 AM filip <filip....@gmail.com> wrote:
>>>
>>> > Is it by design that the schema evolution API for adding top-level
>>> fields
>>> > will always create an optional field as per SchemaUpdate code [1]?
>>> >
>>> > [1]
>>> >
>>> >
>>> https://github.com/Netflix/iceberg/blob/master/core/src/main/java/com/netflix/iceberg/SchemaUpdate.java#L102
>>> >
>>> > --
>>> > Filip Bocse
>>> >
>>>
>>>
>>> --
>>> Ryan Blue
>>> Software Engineer
>>> Netflix
>>>
>>
>>
>> --
>> Filip Bocse
>>
>
>
> --
> Ryan Blue
> Software Engineer
> Netflix
>

Reply via email to