You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/05/07 21:02:11 UTC

[GitHub] [iceberg] yyanyy commented on a change in pull request #2556: Core: update schema constructor callers to include fresh identifier

yyanyy commented on a change in pull request #2556:
URL: https://github.com/apache/iceberg/pull/2556#discussion_r628519164



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -213,7 +224,7 @@ public static Schema assignIncreasingFreshIds(Schema schema) {
    */
   public static Schema reassignIds(Schema schema, Schema idSourceSchema) {
     Types.StructType struct = visit(schema, new ReassignIds(idSourceSchema)).asStructType();
-    return new Schema(struct.fields());
+    return new Schema(struct.fields(), refreshIdentifierFields(struct, schema));

Review comment:
       since `schema` passed in to this method is almost always a schema just got constructed (and thus have to call this method to assign the right ids), I think `refreshIdentifierFields(struct, schema)` here will almost always be a no-op? 
   
   I wonder if we want to use `idSourceSchema` to get the identifier fields, although that might have a different problem that the input `schema` could be a subset of `idSourceSchema` and thus doesn't include all identifier fields. Though in this case identifier fields won't be useful and we may skip it, and we can visit two schema and verify they have the same number of columns to identify this case. I'm not sure about the use case of this method and if we really need to assign identifier fields here though. 

##########
File path: api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java
##########
@@ -42,6 +44,44 @@ public void testReassignIdsDuplicateColumns() {
     Assert.assertEquals(sourceSchema.asStruct(), actualSchema.asStruct());
   }
 
+  @Test
+  public void testReassignIdsWithIdentifier() {
+    Schema schema = new Schema(
+        Lists.newArrayList(
+            required(0, "a", Types.IntegerType.get()),
+            required(1, "A", Types.IntegerType.get())),
+        Sets.newHashSet(0)
+    );
+    Schema sourceSchema = new Schema(
+        Lists.newArrayList(
+            required(1, "a", Types.IntegerType.get()),
+            required(2, "A", Types.IntegerType.get())),
+        Sets.newHashSet(1)
+    );
+    final Schema actualSchema = TypeUtil.reassignIds(schema, sourceSchema);
+    Assert.assertEquals(sourceSchema.asStruct(), actualSchema.asStruct());
+    Assert.assertEquals(sourceSchema.identifierFieldIds(), actualSchema.identifierFieldIds());
+  }
+
+  @Test
+  public void testAssignIncreasingFreshIdWithIdentifier() {
+    Schema schema = new Schema(
+        Lists.newArrayList(
+            required(10, "a", Types.IntegerType.get()),
+            required(11, "A", Types.IntegerType.get())),
+        Sets.newHashSet(10)
+    );
+    Schema sourceSchema = new Schema(
+        Lists.newArrayList(
+            required(1, "a", Types.IntegerType.get()),
+            required(2, "A", Types.IntegerType.get())),
+        Sets.newHashSet(1)
+    );
+    final Schema actualSchema = TypeUtil.assignIncreasingFreshIds(schema);
+    Assert.assertEquals(sourceSchema.asStruct(), actualSchema.asStruct());
+    Assert.assertEquals(sourceSchema.identifierFieldIds(), actualSchema.identifierFieldIds());
+  }
+

Review comment:
       do we want to add a case to call `assignFreshIds(Schema schema, Schema baseSchema, NextID nextId)` with `schema` doesn't have identifier fields but `baseSchema` has, and the output schema will have identifier fields? I think it's an existing use case we have in TableMetadata that may worth explicit testing. 




-- 
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.

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



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