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/09 02:42:22 UTC

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

jun-he commented on a change in pull request #2284:
URL: https://github.com/apache/iceberg/pull/2284#discussion_r628823549



##########
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:
       As mentioned in https://github.com/apache/iceberg/pull/2284#discussion_r609381706, the issue to replace old one using void transform is that the table schema might be changed and cannot create void transform if the old source field is removed.
   @rdblue  do you think we should use a dummy source field in this case?




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