You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by fo...@apache.org on 2023/06/27 15:01:19 UTC

[iceberg] branch master updated: Nessie: Minor Refactoring of NessieTableOperations (#7893)

This is an automated email from the ASF dual-hosted git repository.

fokko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new dd0ee24860 Nessie: Minor Refactoring of NessieTableOperations (#7893)
dd0ee24860 is described below

commit dd0ee24860a0080c5eff85fdcf4c6b48a0e5e84e
Author: Ajantha Bhat <aj...@gmail.com>
AuthorDate: Tue Jun 27 20:31:13 2023 +0530

    Nessie: Minor Refactoring of NessieTableOperations (#7893)
    
    * Nessie: Minor Refactoring of NessieTableOperations
    
    * Handle comments
    
    * Handle IO outisde the NessieUtil
---
 .../apache/iceberg/nessie/NessieIcebergClient.java |  6 ++-
 .../iceberg/nessie/NessieTableOperations.java      | 56 ++++-----------------
 .../java/org/apache/iceberg/nessie/NessieUtil.java | 58 ++++++++++++++++++++++
 3 files changed, 74 insertions(+), 46 deletions(-)

diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java
index 8b17194acb..e64414e999 100644
--- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java
+++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java
@@ -85,10 +85,14 @@ public class NessieIcebergClient implements AutoCloseable {
     return api;
   }
 
-  public UpdateableReference getRef() {
+  UpdateableReference getRef() {
     return reference.get();
   }
 
+  public Reference getReference() {
+    return reference.get().getReference();
+  }
+
   public void refresh() throws NessieNotFoundException {
     getRef().refresh(api);
   }
diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
index 43a453c78a..863f90b3b5 100644
--- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
+++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java
@@ -21,10 +21,8 @@ package org.apache.iceberg.nessie;
 import java.util.List;
 import java.util.Map;
 import org.apache.iceberg.BaseMetastoreTableOperations;
-import org.apache.iceberg.SnapshotRef;
 import org.apache.iceberg.TableMetadata;
 import org.apache.iceberg.TableMetadataParser;
-import org.apache.iceberg.TableProperties;
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.CommitStateUnknownException;
@@ -32,7 +30,6 @@ import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
 import org.apache.iceberg.exceptions.NoSuchNamespaceException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
 import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.projectnessie.client.http.HttpClientException;
 import org.projectnessie.error.NessieConflictException;
 import org.projectnessie.error.NessieNotFoundException;
@@ -81,47 +78,6 @@ public class NessieTableOperations extends BaseMetastoreTableOperations {
     return key.toString();
   }
 
-  private TableMetadata loadTableMetadata(String metadataLocation, Reference reference) {
-    // Update the TableMetadata with the Content of NessieTableState.
-    TableMetadata deserialized = TableMetadataParser.read(io(), metadataLocation);
-    Map<String, String> newProperties = Maps.newHashMap(deserialized.properties());
-    newProperties.put(NESSIE_COMMIT_ID_PROPERTY, reference.getHash());
-    // To prevent accidental deletion of files that are still referenced by other branches/tags,
-    // setting GC_ENABLED to false. So that all Iceberg's gc operations like expire_snapshots,
-    // remove_orphan_files, drop_table with purge will fail with an error.
-    // Nessie CLI will provide a reference aware GC functionality for the expired/unreferenced
-    // files.
-    newProperties.put(TableProperties.GC_ENABLED, "false");
-
-    boolean metadataCleanupEnabled =
-        newProperties
-            .getOrDefault(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false")
-            .equalsIgnoreCase("true");
-    if (metadataCleanupEnabled) {
-      newProperties.put(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false");
-      LOG.warn(
-          "Automatic table metadata files cleanup was requested, but disabled because "
-              + "the Nessie catalog can use historical metadata files from other references. "
-              + "Use the 'nessie-gc' tool for history-aware GC");
-    }
-
-    TableMetadata.Builder builder =
-        TableMetadata.buildFrom(deserialized)
-            .setPreviousFileLocation(null)
-            .setCurrentSchema(table.getSchemaId())
-            .setDefaultSortOrder(table.getSortOrderId())
-            .setDefaultPartitionSpec(table.getSpecId())
-            .withMetadataLocation(metadataLocation)
-            .setProperties(newProperties);
-    if (table.getSnapshotId() != -1) {
-      builder.setBranchSnapshot(table.getSnapshotId(), SnapshotRef.MAIN_BRANCH);
-    }
-    LOG.info(
-        "loadTableMetadata for '{}' from location '{}' at '{}'", key, metadataLocation, reference);
-
-    return builder.discardChanges().build();
-  }
-
   @Override
   protected void doRefresh() {
     try {
@@ -159,7 +115,17 @@ public class NessieTableOperations extends BaseMetastoreTableOperations {
         throw new NoSuchTableException(ex, "No such table '%s'", key);
       }
     }
-    refreshFromMetadataLocation(metadataLocation, null, 2, l -> loadTableMetadata(l, reference));
+    refreshFromMetadataLocation(
+        metadataLocation,
+        null,
+        2,
+        location ->
+            NessieUtil.updateTableMetadataWithNessieSpecificProperties(
+                TableMetadataParser.read(fileIO, location),
+                location,
+                table,
+                key.toString(),
+                reference));
   }
 
   @Override
diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java
index 0420597945..4a7a73e408 100644
--- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java
+++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java
@@ -24,16 +24,26 @@ import java.util.Map;
 import java.util.Optional;
 import javax.annotation.Nullable;
 import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.projectnessie.model.CommitMeta;
 import org.projectnessie.model.ContentKey;
+import org.projectnessie.model.IcebergTable;
 import org.projectnessie.model.ImmutableCommitMeta;
+import org.projectnessie.model.Reference;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public final class NessieUtil {
 
+  private static final Logger LOG = LoggerFactory.getLogger(NessieUtil.class);
+
   public static final String NESSIE_CONFIG_PREFIX = "nessie.";
   static final String APPLICATION_TYPE = "application-type";
 
@@ -91,4 +101,52 @@ public final class NessieUtil {
     return Optional.ofNullable(catalogOptions.get(CatalogProperties.USER))
         .orElseGet(() -> System.getProperty("user.name"));
   }
+
+  public static TableMetadata updateTableMetadataWithNessieSpecificProperties(
+      TableMetadata tableMetadata,
+      String metadataLocation,
+      IcebergTable table,
+      String identifier,
+      Reference reference) {
+    // Update the TableMetadata with the Content of NessieTableState.
+    Map<String, String> newProperties = Maps.newHashMap(tableMetadata.properties());
+    newProperties.put(NessieTableOperations.NESSIE_COMMIT_ID_PROPERTY, reference.getHash());
+    // To prevent accidental deletion of files that are still referenced by other branches/tags,
+    // setting GC_ENABLED to false. So that all Iceberg's gc operations like expire_snapshots,
+    // remove_orphan_files, drop_table with purge will fail with an error.
+    // Nessie CLI will provide a reference aware GC functionality for the expired/unreferenced
+    // files.
+    newProperties.put(TableProperties.GC_ENABLED, "false");
+
+    boolean metadataCleanupEnabled =
+        newProperties
+            .getOrDefault(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false")
+            .equalsIgnoreCase("true");
+    if (metadataCleanupEnabled) {
+      newProperties.put(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false");
+      LOG.warn(
+          "Automatic table metadata files cleanup was requested, but disabled because "
+              + "the Nessie catalog can use historical metadata files from other references. "
+              + "Use the 'nessie-gc' tool for history-aware GC");
+    }
+
+    TableMetadata.Builder builder =
+        TableMetadata.buildFrom(tableMetadata)
+            .setPreviousFileLocation(null)
+            .setCurrentSchema(table.getSchemaId())
+            .setDefaultSortOrder(table.getSortOrderId())
+            .setDefaultPartitionSpec(table.getSpecId())
+            .withMetadataLocation(metadataLocation)
+            .setProperties(newProperties);
+    if (table.getSnapshotId() != -1) {
+      builder.setBranchSnapshot(table.getSnapshotId(), SnapshotRef.MAIN_BRANCH);
+    }
+    LOG.info(
+        "loadTableMetadata for '{}' from location '{}' at '{}'",
+        identifier,
+        metadataLocation,
+        reference);
+
+    return builder.discardChanges().build();
+  }
 }