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 2022/02/10 02:00:52 UTC

[GitHub] [iceberg] amogh-jahagirdar opened a new pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

amogh-jahagirdar opened a new pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082


   To help unblock any development on TableMetadata Builder API changes, we can add the reference mapping structure to TableMetadata (and perform basic validation on this structure).
   
   Note in this PR we aren't populating references from the metadata file (did not integrate the new SnapshotRefParser changes). We can have that in this PR or a follow on. For now I left it off (and we won't attempt to parse the snapshot ref metadata). But I can add that in this PR if we feel it is better.


-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803308200



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       > If you remove main then current-snapshot-id is -1 but we don't actually have -1 in this map?
   
   I remember we were thinking about deprecating `current-snapshot-id` after the branch change, because `current-snapshot-id` should now be retrieved by getting the snapshot ID of the main branch, and that could be -1 if not exist. If we go with that route, we allow a `{"main": -1}` entry in the map and we can deprecate `current-snapshot-id`. If we keep `current-snapshot-id`, then that information of `{"main": -1}` is duplicate and we don't need to keep that. Either way is fine to me, do you think any one of them has any particular advantage?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] rdblue commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803262740



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadataParser.java
##########
@@ -433,6 +434,7 @@ static TableMetadata fromJson(FileIO io, String metadataLocation, JsonNode node)
     return new TableMetadata(metadataLocation, formatVersion, uuid, location,
         lastSequenceNumber, lastUpdatedMillis, lastAssignedColumnId, currentSchemaId, schemas, defaultSpecId, specs,
         lastAssignedPartitionId, defaultSortOrderId, sortOrders, properties, currentVersionId,
-        snapshots, entries.build(), metadataEntries.build(), ImmutableList.of() /* no changes from the file */);
+        snapshots, entries.build(), metadataEntries.build(), ImmutableMap.of(), ImmutableList.of() /* no changes from
+         the file */);

Review comment:
       Can you wrap the line on the argument, rather than in the middle of a comment?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803308200



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       > If you remove main then current-snapshot-id is -1 but we don't actually have -1 in this map?
   
   I remember we were thinking about deprecating `current-snapshot-id` after branching is added to spec, because `current-snapshot-id` should now be retrieved by getting the snapshot ID of the main branch, and that could be -1 if not exist. If we go with that route, we allow a `{"main": -1}` entry in the map and we can deprecate `current-snapshot-id`. If we keep `current-snapshot-id`, then that information of `{"main": -1}` is duplicate and we don't need to keep that. Either way is fine to me, do you think any one of them has any particular advantage?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] amogh-jahagirdar commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803250465



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateAndCompleteRefs(Map<String, SnapshotRef> inputRefs) {

Review comment:
       Ah yes. There is no "completion" as such, since we're always assuming that main exists. I think it could be worth at the end of validating the references in a loop, we perform one final validation that main reference exists in the metadata. Otherwise we fail. Regardless, it is still validation so just "validateRefs" should be good.
   
   I was going to do this in this PR but that breaks a bunch of tests which rely on tableMetadata as the main reference isn't defined in their metadata. Once we get some more clarity though I can add that validation and update the tests. Thoughts?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] amogh-jahagirdar commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803290070



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       Removed the need for a special error message.
   
   Will let @jackye1995 comment, but would we be putting any implications (i.e. potentially unwanted behaviors) on retention if we allowed main to be dropped? If we allow main to be dropped this means that we must have a fallback mechanism for any other reference types (which would be the retention already defined at the table level). 
   
   This seems like it should be okay, but just trying to think through if there are any cases where allowing someone to drop main leads to any undesired behavior in snapshot expiration.




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] amogh-jahagirdar commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803250465



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateAndCompleteRefs(Map<String, SnapshotRef> inputRefs) {

Review comment:
       Ah yes. There is no "completion" as such, since we're always assuming that main exists. I think it could be worth at the end of validating the references in a loop, we perform one final validation that main reference exists in the metadata. Otherwise we fail since the metadata is basically in a corrupted state at that point. Regardless, it is still validation so just "validateRefs" should be good.
   
   I was going to do this in this PR but that breaks a bunch of tests which rely on tableMetadata as the main reference isn't defined in their metadata. Once we get some more clarity though I can add that validation and update the tests. Thoughts?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] rdblue merged pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082


   


-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803308200



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       > If you remove main then current-snapshot-id is -1 but we don't actually have -1 in this map?
   
   I remember we were thinking about deprecating `current-snapshot-id` after branching is added to spec, because `current-snapshot-id` should now be retrieved by getting the snapshot ID of the main branch, and that could be -1 if not exist. If we go with that route, `main` is always an entry in the map, and we allow a `{"main": -1}` entry in the map and we can deprecate `current-snapshot-id`. If we keep `current-snapshot-id`, then that information of `{"main": -1}` is duplicate and we don't need to keep that. Either way is fine to me, do you think any one of them has any particular advantage?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] amogh-jahagirdar commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803290070



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       Removed the need for a special error message.
   
   Will let @jackye1995 comment, but would we be putting any implications (i.e. potentially unwanted behaviors) on retention if we allowed main to be dropped? If we allow main to be dropped this means that we must have a fallback mechanism for any other reference types (which would be the retention already defined at the table level). 
   
   This seems like it should be okay, but just trying to think through if there are any cases where allowing someone to drop main leads to any undesired behavior in retention.




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] rdblue commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803831394



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       I agree. We should not explicitly track -1, but when the current snapshot is requested we can return null if there is no `main`. And we should keep `current-snapshot-id` up to date for older readers.




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] amogh-jahagirdar commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803290070



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       Removed the special error message case.
   
   Will let @jackye1995 comment, but would we be putting any implications (i.e. potentially unwanted behaviors) on retention if we allowed main to be dropped? If we allow main to be dropped this means that we must have a fallback mechanism for any other reference types (which would be the retention already defined at the table level). 
   
   This seems like it should be okay, but just trying to think through if there are any cases where allowing someone to drop main leads to any undesired behavior in snapshot expiration.




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] rdblue commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803263738



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       I think this should be `if (ref.snapshotId() == -1 && refName.equals(MAIN_BRANCH))` then ignore it. Otherwise, check the other precondition. No need to have a special error message for -1 that states main can be -1.
   
   Also, for @jackye1995, do we want to just allow main to be dropped? If you remove main then current-snapshot-id is -1 but we don't actually have -1 in this map?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803311605



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateRefs(Map<String, SnapshotRef> inputRefs) {
+    for (Map.Entry<String, SnapshotRef> refEntry : inputRefs.entrySet()) {
+      String refName = refEntry.getKey();
+      SnapshotRef ref = refEntry.getValue();
+      if (ref.snapshotId() == -1) {
+        Preconditions.checkArgument(refName.equals(SnapshotRef.MAIN_BRANCH),
+            "Snapshot ref %s refers to snapshot id -1. Only main can refer to -1", refName);

Review comment:
       keeping `current-snapshot-id` has the advantage that an old reader can continue to read the new metadata and get current snapshot id. In that sense, we will likely not be able to retire the field `current-snapshot-id`. So we should probably go with the logic of not having `main` in the map, and if people do `table.ref("main")` it should return `table.currentSnapshot().snapshotId()` or -1




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] amogh-jahagirdar commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803277723



##########
File path: core/src/test/java/org/apache/iceberg/TestTableMetadata.java
##########
@@ -108,7 +108,8 @@ public void testJsonConversion() throws Exception {
         7, ImmutableList.of(TEST_SCHEMA, schema),
         5, ImmutableList.of(SPEC_5), SPEC_5.lastAssignedFieldId(),
         3, ImmutableList.of(SORT_ORDER_3), ImmutableMap.of("property", "value"), currentSnapshotId,
-        Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of(), ImmutableList.of());
+        Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of(),
+        ImmutableMap.of(), ImmutableList.of());

Review comment:
       No, it should be fixed, only the last arg should be on a new line




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] amogh-jahagirdar commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
amogh-jahagirdar commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803277723



##########
File path: core/src/test/java/org/apache/iceberg/TestTableMetadata.java
##########
@@ -108,7 +108,8 @@ public void testJsonConversion() throws Exception {
         7, ImmutableList.of(TEST_SCHEMA, schema),
         5, ImmutableList.of(SPEC_5), SPEC_5.lastAssignedFieldId(),
         3, ImmutableList.of(SORT_ORDER_3), ImmutableMap.of("property", "value"), currentSnapshotId,
-        Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of(), ImmutableList.of());
+        Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of(),
+        ImmutableMap.of(), ImmutableList.of());

Review comment:
       Ah no, it should be fixed, only the last arg should be on a new line




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] rdblue commented on pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#issuecomment-1035093823


   Thanks for getting this in so quickly, @amogh-jahagirdar! Nice work.


-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803248939



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -725,6 +728,21 @@ private static SortOrder freshSortOrder(int orderId, Schema schema, SortOrder so
     return builder.build();
   }
 
+  private Map<String, SnapshotRef> validateAndCompleteRefs(Map<String, SnapshotRef> inputRefs) {

Review comment:
       looks like we are just validating refs, should this method name be updated?




-- 
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-unsubscribe@iceberg.apache.org

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


[GitHub] [iceberg] rdblue commented on a change in pull request #4082: Core: Add reference structures to TableMetadata to unblock TableMetadata API changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #4082:
URL: https://github.com/apache/iceberg/pull/4082#discussion_r803262862



##########
File path: core/src/test/java/org/apache/iceberg/TestTableMetadata.java
##########
@@ -108,7 +108,8 @@ public void testJsonConversion() throws Exception {
         7, ImmutableList.of(TEST_SCHEMA, schema),
         5, ImmutableList.of(SPEC_5), SPEC_5.lastAssignedFieldId(),
         3, ImmutableList.of(SORT_ORDER_3), ImmutableMap.of("property", "value"), currentSnapshotId,
-        Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of(), ImmutableList.of());
+        Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of(),
+        ImmutableMap.of(), ImmutableList.of());

Review comment:
       is this line wrapping correct?




-- 
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-unsubscribe@iceberg.apache.org

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