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/04 00:00:41 UTC

[GitHub] [iceberg] rdblue commented on a change in pull request #2284: Core: reassign the partition field IDs and reuse any existing IDs

rdblue commented on a change in pull request #2284:
URL: https://github.com/apache/iceberg/pull/2284#discussion_r625438418



##########
File path: core/src/main/java/org/apache/iceberg/TableMetadata.java
##########
@@ -703,6 +705,35 @@ public TableMetadata removeSnapshotLogEntries(Set<Long> snapshotIds) {
         snapshots, newSnapshotLog, addPreviousFile(file, lastUpdatedMillis));
   }
 
+  private PartitionSpec reassignPartitionIds(PartitionSpec partitionSpec, AtomicInteger lastPartitionId) {
+    if (formatVersion > 1) {
+      Map<Pair<Integer, String>, Integer> transformToFieldId = specs.stream()
+          .flatMap(spec -> spec.fields().stream())
+          .collect(Collectors.toMap(
+              field -> Pair.of(field.sourceId(), field.transform().toString()),
+              PartitionField::fieldId,
+              (n1, n2) -> n2));
+
+      PartitionSpec.Builder specBuilder = PartitionSpec.builderFor(partitionSpec.schema())
+          .withSpecId(partitionSpec.specId());
+
+      for (PartitionField field : partitionSpec.fields()) {
+        // reassign the partition field ids
+        Integer fieldId = transformToFieldId.computeIfAbsent(
+            Pair.of(field.sourceId(), field.transform().toString()), k -> lastPartitionId.incrementAndGet());
+        specBuilder.add(
+            field.sourceId(),
+            fieldId,
+            field.name(),
+            field.transform());
+      }
+      return specBuilder.build();
+    } else {
+      // noop for v1 table
+      return partitionSpec;

Review comment:
       I think the algorithm for v1 tables should be to find out what matches previous specs and re-use those. Then if there is a new field it is moved to the end, and if there is an old field that is unmatched it gets replaced with a `void` transform. Otherwise, this will create partition field ID conflicts in v1 tables.




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