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

emaynard pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/polaris.git


The following commit(s) were added to refs/heads/main by this push:
     new a5552a186 Implement GenericTableCatalog (#1231)
a5552a186 is described below

commit a5552a18634ce67d3f21852a3ff15e4f0f02e575
Author: Eric Maynard <[email protected]>
AuthorDate: Wed Mar 26 13:45:06 2025 -0700

    Implement GenericTableCatalog (#1231)
    
    * add missing apis
    
    * more tests, fixes
    
    * clean up drop
    
    * autolint
    
    * changes per review
    
    * revert iceberg messages to comply with oss tests
    
    * another revert
    
    * more iceberg catalog changes
    
    * autolint
    
    * wip
    
    * refactor to subtype
    
    * autolint
    
    * rebase
    
    * add another assert
    
    * autolint
    
    * add another best effort check
    
    * autolint
    
    * reduce metastore trips
    
    * autolint
---
 .../polaris/core/entity/PolarisEntitySubType.java  |   5 +-
 .../polaris/core/entity/PolarisEntityType.java     |   8 +-
 .../polaris/core/entity/PolarisPrivilege.java      |  28 +--
 .../entity/{ => table}/GenericTableEntity.java     |  17 +-
 .../entity/{ => table}/IcebergTableLikeEntity.java |  33 +--
 .../polaris/core/entity/table/TableLikeEntity.java |  56 +++++
 .../core/persistence/resolver/Resolver.java        |   3 +-
 .../DataCompactionPolicyValidator.java             |   6 +-
 .../storage/PolarisStorageConfigurationInfo.java   |   2 +-
 .../core/storage/cache/StorageCredentialCache.java |   2 +-
 .../polaris/core/persistence/EntityCacheTest.java  |   9 +-
 .../polaris/core/persistence/ResolverTest.java     |  17 +-
 .../DataCompactionPolicyValidatorTest.java         |  10 +-
 .../storage/cache/StorageCredentialCacheTest.java  |  12 +-
 .../persistence/PolarisTestMetaStoreManager.java   | 148 ++++++------
 .../quarkus/catalog/GenericTableCatalogTest.java   | 264 ++++++++++++++++++++-
 .../quarkus/catalog/IcebergCatalogTest.java        |   2 +-
 .../quarkus/task/TableCleanupTaskHandlerTest.java  |   2 +-
 .../polaris/service/admin/PolarisAdminService.java |  43 ++--
 .../catalog/generic/GenericTableCatalog.java       |  56 ++++-
 .../service/catalog/iceberg/IcebergCatalog.java    |  87 +++----
 .../iceberg/IcebergCatalogHandlerWrapper.java      |  70 +++---
 .../service/task/TableCleanupTaskHandler.java      |   4 +-
 .../polaris/service/task/TaskFileIOSupplier.java   |   2 +-
 24 files changed, 616 insertions(+), 270 deletions(-)

diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntitySubType.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntitySubType.java
index e45f4d910..2d2c666cb 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntitySubType.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntitySubType.java
@@ -30,8 +30,9 @@ public enum PolarisEntitySubType {
   ANY_SUBTYPE(-1, null),
   // the NULL value is used when an entity has no subtype, i.e. NOT_APPLICABLE 
really
   NULL_SUBTYPE(0, null),
-  TABLE(2, PolarisEntityType.ICEBERG_TABLE_LIKE),
-  VIEW(3, PolarisEntityType.ICEBERG_TABLE_LIKE);
+  ICEBERG_TABLE(2, PolarisEntityType.TABLE_LIKE),
+  ICEBERG_VIEW(3, PolarisEntityType.TABLE_LIKE),
+  GENERIC_TABLE(4, PolarisEntityType.TABLE_LIKE);
 
   // to efficiently map the code of a subtype to its corresponding subtype 
enum, use a reverse
   // array which is initialized below
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntityType.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntityType.java
index 90d9aa9e5..fae2d66eb 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntityType.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisEntityType.java
@@ -31,12 +31,10 @@ public enum PolarisEntityType {
   CATALOG(4, ROOT, false, false),
   CATALOG_ROLE(5, CATALOG, true, false),
   NAMESPACE(6, CATALOG, false, true),
-  // generic table is either a view or a real table
-  ICEBERG_TABLE_LIKE(7, NAMESPACE, false, false),
+  TABLE_LIKE(7, NAMESPACE, false, false),
   TASK(8, ROOT, false, false),
-  FILE(9, ICEBERG_TABLE_LIKE, false, false),
-  POLICY(10, NAMESPACE, false, false),
-  GENERIC_TABLE(11, NAMESPACE, false, false);
+  FILE(9, TABLE_LIKE, false, false),
+  POLICY(10, NAMESPACE, false, false);
 
   // to efficiently map a code to its corresponding entity type, use a reverse 
array which
   // is initialized below
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisPrivilege.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisPrivilege.java
index 122d39a4c..8fb0990e2 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisPrivilege.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/PolarisPrivilege.java
@@ -41,22 +41,22 @@ public enum PolarisPrivilege {
   TABLE_CREATE(6, PolarisEntityType.NAMESPACE),
   VIEW_CREATE(7, PolarisEntityType.NAMESPACE),
   NAMESPACE_DROP(8, PolarisEntityType.NAMESPACE),
-  TABLE_DROP(9, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE),
-  VIEW_DROP(10, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW),
+  TABLE_DROP(9, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE),
+  VIEW_DROP(10, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_VIEW),
   NAMESPACE_LIST(11, PolarisEntityType.NAMESPACE),
   TABLE_LIST(12, PolarisEntityType.NAMESPACE),
   VIEW_LIST(13, PolarisEntityType.NAMESPACE),
   NAMESPACE_READ_PROPERTIES(14, PolarisEntityType.NAMESPACE),
-  TABLE_READ_PROPERTIES(15, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE),
-  VIEW_READ_PROPERTIES(16, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW),
+  TABLE_READ_PROPERTIES(15, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE),
+  VIEW_READ_PROPERTIES(16, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_VIEW),
   NAMESPACE_WRITE_PROPERTIES(17, PolarisEntityType.NAMESPACE),
-  TABLE_WRITE_PROPERTIES(18, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE),
-  VIEW_WRITE_PROPERTIES(19, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW),
-  TABLE_READ_DATA(20, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE),
-  TABLE_WRITE_DATA(21, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE),
+  TABLE_WRITE_PROPERTIES(18, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE),
+  VIEW_WRITE_PROPERTIES(19, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_VIEW),
+  TABLE_READ_DATA(20, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE),
+  TABLE_WRITE_DATA(21, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE),
   NAMESPACE_FULL_METADATA(22, PolarisEntityType.NAMESPACE),
-  TABLE_FULL_METADATA(23, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE),
-  VIEW_FULL_METADATA(24, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW),
+  TABLE_FULL_METADATA(23, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE),
+  VIEW_FULL_METADATA(24, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_VIEW),
   CATALOG_CREATE(25, PolarisEntityType.ROOT),
   CATALOG_DROP(26, PolarisEntityType.CATALOG),
   CATALOG_LIST(27, PolarisEntityType.ROOT),
@@ -70,14 +70,14 @@ public enum PolarisPrivilege {
   CATALOG_ROLE_LIST_GRANTS(35, PolarisEntityType.PRINCIPAL),
   CATALOG_LIST_GRANTS(36, PolarisEntityType.CATALOG),
   NAMESPACE_LIST_GRANTS(37, PolarisEntityType.NAMESPACE),
-  TABLE_LIST_GRANTS(38, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE),
-  VIEW_LIST_GRANTS(39, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW),
+  TABLE_LIST_GRANTS(38, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE),
+  VIEW_LIST_GRANTS(39, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_VIEW),
   CATALOG_MANAGE_GRANTS_ON_SECURABLE(40, PolarisEntityType.CATALOG),
   NAMESPACE_MANAGE_GRANTS_ON_SECURABLE(41, PolarisEntityType.NAMESPACE),
   TABLE_MANAGE_GRANTS_ON_SECURABLE(
-      42, PolarisEntityType.ICEBERG_TABLE_LIKE, PolarisEntitySubType.TABLE),
+      42, PolarisEntityType.TABLE_LIKE, PolarisEntitySubType.ICEBERG_TABLE),
   VIEW_MANAGE_GRANTS_ON_SECURABLE(
-      43, PolarisEntityType.ICEBERG_TABLE_LIKE, PolarisEntitySubType.VIEW),
+      43, PolarisEntityType.TABLE_LIKE, PolarisEntitySubType.ICEBERG_VIEW),
   PRINCIPAL_CREATE(44, PolarisEntityType.ROOT),
   PRINCIPAL_DROP(45, PolarisEntityType.PRINCIPAL),
   PRINCIPAL_LIST(46, PolarisEntityType.ROOT),
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/GenericTableEntity.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/table/GenericTableEntity.java
similarity index 80%
rename from 
polaris-core/src/main/java/org/apache/polaris/core/entity/GenericTableEntity.java
rename to 
polaris-core/src/main/java/org/apache/polaris/core/entity/table/GenericTableEntity.java
index 5553b1a46..5730ebe0e 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/GenericTableEntity.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/table/GenericTableEntity.java
@@ -16,19 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.polaris.core.entity;
+package org.apache.polaris.core.entity.table;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.rest.RESTUtil;
+import org.apache.polaris.core.entity.NamespaceEntity;
+import org.apache.polaris.core.entity.PolarisBaseEntity;
+import org.apache.polaris.core.entity.PolarisEntity;
+import org.apache.polaris.core.entity.PolarisEntitySubType;
+import org.apache.polaris.core.entity.PolarisEntityType;
 
 /**
- * A {@link PolarisEntity} implementation for generic tables. These tables are 
not Iceberg-like in
- * that they may not have a schema or base location. Similarly to {@link 
IcebergTableLikeEntity}
- * however, these tables have an identifier and a parent namespace.
+ * A {@link TableLikeEntity} implementation for generic tables. These tables 
are not Iceberg-like in
+ * that they may not have a schema or base location.
  */
-public class GenericTableEntity extends PolarisEntity {
+public class GenericTableEntity extends TableLikeEntity {
 
   public static final String FORMAT_KEY = "format";
 
@@ -52,7 +56,8 @@ public class GenericTableEntity extends PolarisEntity {
       extends PolarisEntity.BaseBuilder<GenericTableEntity, 
GenericTableEntity.Builder> {
     public Builder(TableIdentifier tableIdentifier, String format) {
       super();
-      setType(PolarisEntityType.GENERIC_TABLE);
+      setType(PolarisEntityType.TABLE_LIKE);
+      setSubType(PolarisEntitySubType.GENERIC_TABLE);
       setTableIdentifier(tableIdentifier);
       setFormat(format);
     }
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/IcebergTableLikeEntity.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/table/IcebergTableLikeEntity.java
similarity index 86%
rename from 
polaris-core/src/main/java/org/apache/polaris/core/entity/IcebergTableLikeEntity.java
rename to 
polaris-core/src/main/java/org/apache/polaris/core/entity/table/IcebergTableLikeEntity.java
index e7e181075..026a569b3 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/IcebergTableLikeEntity.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/table/IcebergTableLikeEntity.java
@@ -16,15 +16,24 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.polaris.core.entity;
+package org.apache.polaris.core.entity.table;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import java.util.Optional;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.rest.RESTUtil;
-
-public class IcebergTableLikeEntity extends PolarisEntity {
+import org.apache.polaris.core.entity.NamespaceEntity;
+import org.apache.polaris.core.entity.PolarisBaseEntity;
+import org.apache.polaris.core.entity.PolarisEntity;
+import org.apache.polaris.core.entity.PolarisEntityConstants;
+import org.apache.polaris.core.entity.PolarisEntityType;
+
+/**
+ * An entity type for {@link TableLikeEntity} instances that conform to 
iceberg semantics around
+ * locations. This includes both Iceberg tables and Iceberg views.
+ */
+public class IcebergTableLikeEntity extends TableLikeEntity {
   // For applicable types, this key on the "internalProperties" map will 
return the location
   // of the internalProperties JSON file.
   public static final String METADATA_LOCATION_KEY = "metadata-location";
@@ -46,22 +55,6 @@ public class IcebergTableLikeEntity extends PolarisEntity {
     return null;
   }
 
-  @JsonIgnore
-  public TableIdentifier getTableIdentifier() {
-    Namespace parent = getParentNamespace();
-    return TableIdentifier.of(parent, getName());
-  }
-
-  @JsonIgnore
-  public Namespace getParentNamespace() {
-    String encodedNamespace =
-        getInternalPropertiesAsMap().get(NamespaceEntity.PARENT_NAMESPACE_KEY);
-    if (encodedNamespace == null) {
-      return Namespace.empty();
-    }
-    return RESTUtil.decodeNamespace(encodedNamespace);
-  }
-
   @JsonIgnore
   public String getMetadataLocation() {
     return getInternalPropertiesAsMap().get(METADATA_LOCATION_KEY);
@@ -82,7 +75,7 @@ public class IcebergTableLikeEntity extends PolarisEntity {
   public static class Builder extends 
PolarisEntity.BaseBuilder<IcebergTableLikeEntity, Builder> {
     public Builder(TableIdentifier identifier, String metadataLocation) {
       super();
-      setType(PolarisEntityType.ICEBERG_TABLE_LIKE);
+      setType(PolarisEntityType.TABLE_LIKE);
       setTableIdentifier(identifier);
       setMetadataLocation(metadataLocation);
     }
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/entity/table/TableLikeEntity.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/table/TableLikeEntity.java
new file mode 100644
index 000000000..22808d1a7
--- /dev/null
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/entity/table/TableLikeEntity.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.polaris.core.entity.table;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import jakarta.annotation.Nonnull;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.rest.RESTUtil;
+import org.apache.polaris.core.entity.NamespaceEntity;
+import org.apache.polaris.core.entity.PolarisBaseEntity;
+import org.apache.polaris.core.entity.PolarisEntity;
+import org.apache.polaris.core.entity.PolarisEntityType;
+
+/**
+ * An entity type for all table-like entities including Iceberg tables, 
Iceberg views, and generic
+ * tables. This entity maps to {@link PolarisEntityType#TABLE_LIKE}
+ */
+public abstract class TableLikeEntity extends PolarisEntity {
+
+  public TableLikeEntity(@Nonnull PolarisBaseEntity sourceEntity) {
+    super(sourceEntity);
+  }
+
+  @JsonIgnore
+  public TableIdentifier getTableIdentifier() {
+    Namespace parent = getParentNamespace();
+    return TableIdentifier.of(parent, getName());
+  }
+
+  @JsonIgnore
+  public Namespace getParentNamespace() {
+    String encodedNamespace =
+        getInternalPropertiesAsMap().get(NamespaceEntity.PARENT_NAMESPACE_KEY);
+    if (encodedNamespace == null) {
+      return Namespace.empty();
+    }
+    return RESTUtil.decodeNamespace(encodedNamespace);
+  }
+}
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java
index 7e152c05f..b1c7fc169 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/persistence/resolver/Resolver.java
@@ -373,8 +373,7 @@ public class Resolver {
 
     // validate input
     diagnostics.check(
-        entityType != PolarisEntityType.NAMESPACE
-            && entityType != PolarisEntityType.ICEBERG_TABLE_LIKE,
+        entityType != PolarisEntityType.NAMESPACE && entityType != 
PolarisEntityType.TABLE_LIKE,
         "cannot_be_path");
     diagnostics.check(
         entityType.isTopLevel() || this.referenceCatalogName != null, 
"reference_catalog_expected");
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/policy/validator/datacompaction/DataCompactionPolicyValidator.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/policy/validator/datacompaction/DataCompactionPolicyValidator.java
index 8ef8b4dcf..344c72d90 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/policy/validator/datacompaction/DataCompactionPolicyValidator.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/policy/validator/datacompaction/DataCompactionPolicyValidator.java
@@ -19,8 +19,8 @@
 package org.apache.polaris.core.policy.validator.datacompaction;
 
 import static org.apache.polaris.core.entity.PolarisEntityType.CATALOG;
-import static 
org.apache.polaris.core.entity.PolarisEntityType.ICEBERG_TABLE_LIKE;
 import static org.apache.polaris.core.entity.PolarisEntityType.NAMESPACE;
+import static org.apache.polaris.core.entity.PolarisEntityType.TABLE_LIKE;
 
 import java.util.Set;
 import org.apache.polaris.core.entity.PolarisEntitySubType;
@@ -32,7 +32,7 @@ public class DataCompactionPolicyValidator implements 
PolicyValidator {
   public static final DataCompactionPolicyValidator INSTANCE = new 
DataCompactionPolicyValidator();
 
   private static final Set<PolarisEntityType> ATTACHABLE_ENTITY_TYPES =
-      Set.of(CATALOG, NAMESPACE, ICEBERG_TABLE_LIKE);
+      Set.of(CATALOG, NAMESPACE, TABLE_LIKE);
 
   @Override
   public void validate(String content) throws InvalidPolicyException {
@@ -49,7 +49,7 @@ public class DataCompactionPolicyValidator implements 
PolicyValidator {
       return false;
     }
 
-    if (entityType == ICEBERG_TABLE_LIKE && entitySubType != 
PolarisEntitySubType.TABLE) {
+    if (entityType == TABLE_LIKE && entitySubType != 
PolarisEntitySubType.ICEBERG_TABLE) {
       return false;
     }
 
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/storage/PolarisStorageConfigurationInfo.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/storage/PolarisStorageConfigurationInfo.java
index 161af2884..e1ec45bdb 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/storage/PolarisStorageConfigurationInfo.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/storage/PolarisStorageConfigurationInfo.java
@@ -41,9 +41,9 @@ import org.apache.polaris.core.admin.model.Catalog;
 import org.apache.polaris.core.config.FeatureConfiguration;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.entity.CatalogEntity;
-import org.apache.polaris.core.entity.IcebergTableLikeEntity;
 import org.apache.polaris.core.entity.PolarisEntity;
 import org.apache.polaris.core.entity.PolarisEntityConstants;
+import org.apache.polaris.core.entity.table.IcebergTableLikeEntity;
 import org.apache.polaris.core.storage.aws.AwsStorageConfigurationInfo;
 import org.apache.polaris.core.storage.azure.AzureStorageConfigurationInfo;
 import org.apache.polaris.core.storage.gcp.GcpStorageConfigurationInfo;
diff --git 
a/polaris-core/src/main/java/org/apache/polaris/core/storage/cache/StorageCredentialCache.java
 
b/polaris-core/src/main/java/org/apache/polaris/core/storage/cache/StorageCredentialCache.java
index 20a371f4e..3fe593195 100644
--- 
a/polaris-core/src/main/java/org/apache/polaris/core/storage/cache/StorageCredentialCache.java
+++ 
b/polaris-core/src/main/java/org/apache/polaris/core/storage/cache/StorageCredentialCache.java
@@ -154,7 +154,7 @@ public class StorageCredentialCache {
   private boolean isTypeSupported(PolarisEntityType type) {
     return type == PolarisEntityType.CATALOG
         || type == PolarisEntityType.NAMESPACE
-        || type == PolarisEntityType.ICEBERG_TABLE_LIKE
+        || type == PolarisEntityType.TABLE_LIKE
         || type == PolarisEntityType.TASK;
   }
 
diff --git 
a/polaris-core/src/test/java/org/apache/polaris/core/persistence/EntityCacheTest.java
 
b/polaris-core/src/test/java/org/apache/polaris/core/persistence/EntityCacheTest.java
index 389f83dea..6e2f458dc 100644
--- 
a/polaris-core/src/test/java/org/apache/polaris/core/persistence/EntityCacheTest.java
+++ 
b/polaris-core/src/test/java/org/apache/polaris/core/persistence/EntityCacheTest.java
@@ -298,8 +298,8 @@ public class EntityCacheTest {
     PolarisBaseEntity T6v1 =
         this.tm.ensureExistsByName(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T6");
     Assertions.assertThat(T6v1).isNotNull();
 
@@ -435,8 +435,7 @@ public class EntityCacheTest {
     Assertions.assertThat(lookup).isNotNull();
 
     EntityCacheByNameKey T4_name =
-        new EntityCacheByNameKey(
-            N1.getCatalogId(), N1.getId(), 
PolarisEntityType.ICEBERG_TABLE_LIKE, "T4");
+        new EntityCacheByNameKey(N1.getCatalogId(), N1.getId(), 
PolarisEntityType.TABLE_LIKE, "T4");
     lookup = cache.getOrLoadEntityByName(callCtx, T4_name);
     Assertions.assertThat(lookup).isNotNull();
     ResolvedPolarisEntity cacheEntry_T4 = lookup.getCacheEntry();
@@ -451,7 +450,7 @@ public class EntityCacheTest {
     // load the renamed entity into cache
     EntityCacheByNameKey T4_renamed =
         new EntityCacheByNameKey(
-            N1.getCatalogId(), N1.getId(), 
PolarisEntityType.ICEBERG_TABLE_LIKE, "T4_renamed");
+            N1.getCatalogId(), N1.getId(), PolarisEntityType.TABLE_LIKE, 
"T4_renamed");
     lookup = cache.getOrLoadEntityByName(callCtx, T4_renamed);
     Assertions.assertThat(lookup).isNotNull();
     ResolvedPolarisEntity cacheEntry_T4_renamed = lookup.getCacheEntry();
diff --git 
a/polaris-core/src/test/java/org/apache/polaris/core/persistence/ResolverTest.java
 
b/polaris-core/src/test/java/org/apache/polaris/core/persistence/ResolverTest.java
index 80bca67af..51ec06a05 100644
--- 
a/polaris-core/src/test/java/org/apache/polaris/core/persistence/ResolverTest.java
+++ 
b/polaris-core/src/test/java/org/apache/polaris/core/persistence/ResolverTest.java
@@ -216,12 +216,12 @@ public class ResolverTest {
 
     // N1/N2/T1 which exists
     ResolverPath N1_N2_T1 =
-        new ResolverPath(List.of("N1", "N2", "T1"), 
PolarisEntityType.ICEBERG_TABLE_LIKE);
+        new ResolverPath(List.of("N1", "N2", "T1"), 
PolarisEntityType.TABLE_LIKE);
     this.resolveDriver(this.cache, "test", N1_N2_T1, null, null);
 
     // N1/N2/T1 which exists
     ResolverPath N1_N2_V1 =
-        new ResolverPath(List.of("N1", "N2", "V1"), 
PolarisEntityType.ICEBERG_TABLE_LIKE);
+        new ResolverPath(List.of("N1", "N2", "V1"), 
PolarisEntityType.TABLE_LIKE);
     this.resolveDriver(this.cache, "test", N1_N2_V1, null, null);
 
     // N5/N6 which exists
@@ -230,7 +230,7 @@ public class ResolverTest {
 
     // N5/N6/T5 which exists
     ResolverPath N5_N6_T5 =
-        new ResolverPath(List.of("N5", "N6", "T5"), 
PolarisEntityType.ICEBERG_TABLE_LIKE);
+        new ResolverPath(List.of("N5", "N6", "T5"), 
PolarisEntityType.TABLE_LIKE);
     this.resolveDriver(this.cache, "test", N5_N6_T5, null, null);
 
     // N7/N8 which exists
@@ -248,7 +248,7 @@ public class ResolverTest {
 
     // Error scenarios: N5/N6/T8 which does not exists
     ResolverPath N5_N6_T8 =
-        new ResolverPath(List.of("N5", "N6", "T8"), 
PolarisEntityType.ICEBERG_TABLE_LIKE);
+        new ResolverPath(List.of("N5", "N6", "T8"), 
PolarisEntityType.TABLE_LIKE);
     this.resolveDriver(
         this.cache,
         "test",
@@ -258,7 +258,7 @@ public class ResolverTest {
 
     // Error scenarios: N8/N6/T8 which does not exists
     ResolverPath N8_N6_T8 =
-        new ResolverPath(List.of("N8", "N6", "T8"), 
PolarisEntityType.ICEBERG_TABLE_LIKE);
+        new ResolverPath(List.of("N8", "N6", "T8"), 
PolarisEntityType.TABLE_LIKE);
     this.resolveDriver(
         this.cache,
         "test",
@@ -277,8 +277,7 @@ public class ResolverTest {
         ResolverStatus.StatusEnum.PATH_COULD_NOT_BE_FULLY_RESOLVED);
 
     // except if the optional flag is specified
-    N5_N6_T8 =
-        new ResolverPath(List.of("N5", "N6", "T8"), 
PolarisEntityType.ICEBERG_TABLE_LIKE, true);
+    N5_N6_T8 = new ResolverPath(List.of("N5", "N6", "T8"), 
PolarisEntityType.TABLE_LIKE, true);
     Resolver resolver =
         this.resolveDriver(this.cache, "test", null, List.of(N1, N5_N6_T8, 
N5_N6_T5, N1_N2), null);
     // get all the resolved paths
@@ -366,7 +365,7 @@ public class ResolverTest {
     ResolverPath N1_N2_PATH = new ResolverPath(List.of("N1", "N2"), 
PolarisEntityType.NAMESPACE);
     this.resolveDriver(this.cache, "test", N1_N2_PATH, null, null);
     ResolverPath N1_N2_T1_PATH =
-        new ResolverPath(List.of("N1", "N2", "T1"), 
PolarisEntityType.ICEBERG_TABLE_LIKE);
+        new ResolverPath(List.of("N1", "N2", "T1"), 
PolarisEntityType.TABLE_LIKE);
     Resolver resolver = this.resolveDriver(this.cache, "test", N1_N2_T1_PATH, 
null, null);
 
     // get the catalog
@@ -400,7 +399,7 @@ public class ResolverTest {
 
     // but we should be able to resolve it under N1/N3
     ResolverPath N1_N3_T1_PATH =
-        new ResolverPath(List.of("N1", "N3", "T1"), 
PolarisEntityType.ICEBERG_TABLE_LIKE);
+        new ResolverPath(List.of("N1", "N3", "T1"), 
PolarisEntityType.TABLE_LIKE);
     this.resolveDriver(this.cache, "test", N1_N3_T1_PATH, null, null);
   }
 
diff --git 
a/polaris-core/src/test/java/org/apache/polaris/core/policy/validator/DataCompactionPolicyValidatorTest.java
 
b/polaris-core/src/test/java/org/apache/polaris/core/policy/validator/DataCompactionPolicyValidatorTest.java
index e833015d6..18205763f 100644
--- 
a/polaris-core/src/test/java/org/apache/polaris/core/policy/validator/DataCompactionPolicyValidatorTest.java
+++ 
b/polaris-core/src/test/java/org/apache/polaris/core/policy/validator/DataCompactionPolicyValidatorTest.java
@@ -19,15 +19,15 @@
 package org.apache.polaris.core.policy.validator;
 
 import static org.apache.polaris.core.entity.PolarisEntitySubType.ANY_SUBTYPE;
-import static org.apache.polaris.core.entity.PolarisEntitySubType.TABLE;
-import static org.apache.polaris.core.entity.PolarisEntitySubType.VIEW;
+import static 
org.apache.polaris.core.entity.PolarisEntitySubType.ICEBERG_TABLE;
+import static org.apache.polaris.core.entity.PolarisEntitySubType.ICEBERG_VIEW;
 import static org.apache.polaris.core.entity.PolarisEntityType.CATALOG;
-import static 
org.apache.polaris.core.entity.PolarisEntityType.ICEBERG_TABLE_LIKE;
 import static org.apache.polaris.core.entity.PolarisEntityType.NAMESPACE;
 import static org.apache.polaris.core.entity.PolarisEntityType.PRINCIPAL;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
+import org.apache.polaris.core.entity.PolarisEntityType;
 import 
org.apache.polaris.core.policy.validator.datacompaction.DataCompactionPolicyValidator;
 import org.junit.jupiter.api.Test;
 
@@ -59,7 +59,7 @@ public class DataCompactionPolicyValidatorTest {
 
   @Test
   public void testCanAttachReturnsTrueForIcebergTableLikeWithTableSubtype() {
-    var result = validator.canAttach(ICEBERG_TABLE_LIKE, TABLE);
+    var result = validator.canAttach(PolarisEntityType.TABLE_LIKE, 
ICEBERG_TABLE);
     assertThat(result)
         .isTrue()
         .as("Expected canAttach() to return true for ICEBERG_TABLE_LIKE with 
TABLE subtype");
@@ -68,7 +68,7 @@ public class DataCompactionPolicyValidatorTest {
   @Test
   public void 
testCanAttachReturnsFalseForIcebergTableLikeWithNonTableSubtype() {
     // For ICEBERG_TABLE_LIKE, any subtype other than TABLE should return 
false.
-    boolean result = validator.canAttach(ICEBERG_TABLE_LIKE, VIEW);
+    boolean result = validator.canAttach(PolarisEntityType.TABLE_LIKE, 
ICEBERG_VIEW);
     assertThat(result)
         .isFalse()
         .as("Expected canAttach() to return false for ICEBERG_TABLE_LIKE with 
non-TABLE subtype");
diff --git 
a/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java
 
b/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java
index f3b4fc2f6..aa22c4218 100644
--- 
a/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java
+++ 
b/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java
@@ -92,7 +92,7 @@ public class StorageCredentialCacheTest {
     PolarisEntity polarisEntity =
         new PolarisEntity(
             new PolarisBaseEntity(
-                1, 2, PolarisEntityType.CATALOG, PolarisEntitySubType.TABLE, 
0, "name"));
+                1, 2, PolarisEntityType.CATALOG, 
PolarisEntitySubType.ICEBERG_TABLE, 0, "name"));
     Assertions.assertThatThrownBy(
             () ->
                 storageCredentialCache.getOrGenerateSubScopeCreds(
@@ -125,7 +125,7 @@ public class StorageCredentialCacheTest {
         .thenReturn(mockedScopedCreds.get(1));
     PolarisBaseEntity baseEntity =
         new PolarisBaseEntity(
-            1, 2, PolarisEntityType.CATALOG, PolarisEntitySubType.TABLE, 0, 
"name");
+            1, 2, PolarisEntityType.CATALOG, 
PolarisEntitySubType.ICEBERG_TABLE, 0, "name");
     PolarisEntity polarisEntity = new PolarisEntity(baseEntity);
 
     // add an item to the cache
@@ -168,7 +168,7 @@ public class StorageCredentialCacheTest {
         .thenReturn(mockedScopedCreds.get(2));
     PolarisBaseEntity baseEntity =
         new PolarisBaseEntity(
-            1, 2, PolarisEntityType.CATALOG, PolarisEntitySubType.TABLE, 0, 
"name");
+            1, 2, PolarisEntityType.CATALOG, 
PolarisEntitySubType.ICEBERG_TABLE, 0, "name");
     PolarisEntity polarisEntity = new PolarisEntity(baseEntity);
     StorageCredentialCacheKey cacheKey =
         new StorageCredentialCacheKey(
@@ -426,15 +426,15 @@ public class StorageCredentialCacheTest {
     PolarisEntity polarisEntity1 =
         new PolarisEntity(
             new PolarisBaseEntity(
-                1, 2, PolarisEntityType.CATALOG, PolarisEntitySubType.TABLE, 
0, "name"));
+                1, 2, PolarisEntityType.CATALOG, 
PolarisEntitySubType.ICEBERG_TABLE, 0, "name"));
     PolarisEntity polarisEntity2 =
         new PolarisEntity(
             new PolarisBaseEntity(
-                2, 2, PolarisEntityType.CATALOG, PolarisEntitySubType.TABLE, 
0, "name"));
+                2, 2, PolarisEntityType.CATALOG, 
PolarisEntitySubType.ICEBERG_TABLE, 0, "name"));
     PolarisEntity polarisEntity3 =
         new PolarisEntity(
             new PolarisBaseEntity(
-                3, 2, PolarisEntityType.CATALOG, PolarisEntitySubType.TABLE, 
0, "name"));
+                3, 2, PolarisEntityType.CATALOG, 
PolarisEntitySubType.ICEBERG_TABLE, 0, "name"));
 
     return Arrays.asList(polarisEntity1, polarisEntity2, polarisEntity3);
   }
diff --git 
a/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/PolarisTestMetaStoreManager.java
 
b/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/PolarisTestMetaStoreManager.java
index b8470dee9..591c657d1 100644
--- 
a/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/PolarisTestMetaStoreManager.java
+++ 
b/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/PolarisTestMetaStoreManager.java
@@ -742,7 +742,7 @@ public class PolarisTestMetaStoreManager {
                   .listEntities(
                       this.polarisCallContext,
                       path,
-                      PolarisEntityType.ICEBERG_TABLE_LIKE,
+                      PolarisEntityType.TABLE_LIKE,
                       PolarisEntitySubType.ANY_SUBTYPE)
                   .getEntities();
           Assertions.assertThat(children).isNotNull();
@@ -1003,35 +1003,35 @@ public class PolarisTestMetaStoreManager {
         this.createEntity(List.of(catalog, N1), PolarisEntityType.NAMESPACE, 
"N2");
     this.createEntity(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T1");
     this.createEntity(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T2");
     this.createEntity(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.VIEW,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_VIEW,
         "V1");
     PolarisBaseEntity N1_N3 =
         this.createEntity(List.of(catalog, N1), PolarisEntityType.NAMESPACE, 
"N3");
     this.createEntity(
         List.of(catalog, N1, N1_N3),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T3");
     this.createEntity(
         List.of(catalog, N1, N1_N3),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.VIEW,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_VIEW,
         "V2");
     this.createEntity(
         List.of(catalog, N1),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T4");
     this.createEntity(List.of(catalog, N1), PolarisEntityType.NAMESPACE, "N4");
     PolarisBaseEntity N5 = this.createEntity(List.of(catalog), 
PolarisEntityType.NAMESPACE, "N5");
@@ -1040,13 +1040,13 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity N5_N6_T5 =
         this.createEntity(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T5");
     this.createEntity(
         List.of(catalog, N5, N5_N6),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T6");
 
     PolarisBaseEntity N7 = this.createEntity(List.of(catalog), 
PolarisEntityType.NAMESPACE, "N7");
@@ -1664,45 +1664,45 @@ public class PolarisTestMetaStoreManager {
         this.ensureExistsByName(List.of(catalog, N1), 
PolarisEntityType.NAMESPACE, "N2");
     this.ensureExistsByName(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T1");
     this.ensureExistsByName(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T2");
     this.ensureExistsByName(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
+        PolarisEntityType.TABLE_LIKE,
         PolarisEntitySubType.ANY_SUBTYPE,
         "T2");
     this.ensureExistsByName(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.VIEW,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_VIEW,
         "V1");
     this.ensureExistsByName(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
+        PolarisEntityType.TABLE_LIKE,
         PolarisEntitySubType.ANY_SUBTYPE,
         "V1");
     PolarisBaseEntity N1_N3 =
         this.ensureExistsByName(List.of(catalog, N1), 
PolarisEntityType.NAMESPACE, "N3");
     this.ensureExistsByName(
         List.of(catalog, N1, N1_N3),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T3");
     this.ensureExistsByName(
         List.of(catalog, N1, N1_N3),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
+        PolarisEntityType.TABLE_LIKE,
         PolarisEntitySubType.ANY_SUBTYPE,
         "V2");
     this.ensureExistsByName(
         List.of(catalog, N1),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T4");
     this.ensureExistsByName(List.of(catalog, N1), PolarisEntityType.NAMESPACE, 
"N4");
     PolarisBaseEntity N5 =
@@ -1715,19 +1715,19 @@ public class PolarisTestMetaStoreManager {
             "N6");
     this.ensureExistsByName(
         List.of(catalog, N5, N5_N6),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T5");
     PolarisBaseEntity N5_N6_T5 =
         this.ensureExistsByName(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
+            PolarisEntityType.TABLE_LIKE,
             PolarisEntitySubType.ANY_SUBTYPE,
             "T5");
     this.ensureExistsByName(
         List.of(catalog, N5, N5_N6),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         "T6");
     PolarisBaseEntity N7 =
         this.ensureExistsByName(List.of(catalog), PolarisEntityType.NAMESPACE, 
"N7");
@@ -1821,9 +1821,9 @@ public class PolarisTestMetaStoreManager {
             ImmutablePair.of("N4", PolarisEntitySubType.NULL_SUBTYPE)));
     this.validateListReturn(
         List.of(catalog, N1),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
+        PolarisEntityType.TABLE_LIKE,
         PolarisEntitySubType.ANY_SUBTYPE,
-        List.of(ImmutablePair.of("T4", PolarisEntitySubType.TABLE)));
+        List.of(ImmutablePair.of("T4", PolarisEntitySubType.ICEBERG_TABLE)));
     PolarisBaseEntity N5 =
         this.ensureExistsByName(List.of(catalog), PolarisEntityType.NAMESPACE, 
"N5");
     this.validateListReturn(
@@ -1837,26 +1837,26 @@ public class PolarisTestMetaStoreManager {
     // table or view object
     this.validateListReturn(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
+        PolarisEntityType.TABLE_LIKE,
         PolarisEntitySubType.ANY_SUBTYPE,
         List.of(
-            ImmutablePair.of("T1", PolarisEntitySubType.TABLE),
-            ImmutablePair.of("T2", PolarisEntitySubType.TABLE),
-            ImmutablePair.of("V1", PolarisEntitySubType.VIEW)));
+            ImmutablePair.of("T1", PolarisEntitySubType.ICEBERG_TABLE),
+            ImmutablePair.of("T2", PolarisEntitySubType.ICEBERG_TABLE),
+            ImmutablePair.of("V1", PolarisEntitySubType.ICEBERG_VIEW)));
     // table object only
     this.validateListReturn(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.TABLE,
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         List.of(
-            ImmutablePair.of("T1", PolarisEntitySubType.TABLE),
-            ImmutablePair.of("T2", PolarisEntitySubType.TABLE)));
+            ImmutablePair.of("T1", PolarisEntitySubType.ICEBERG_TABLE),
+            ImmutablePair.of("T2", PolarisEntitySubType.ICEBERG_TABLE)));
     // view object only
     this.validateListReturn(
         List.of(catalog, N1, N1_N2),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        PolarisEntitySubType.VIEW,
-        List.of(ImmutablePair.of("V1", PolarisEntitySubType.VIEW)));
+        PolarisEntityType.TABLE_LIKE,
+        PolarisEntitySubType.ICEBERG_VIEW,
+        List.of(ImmutablePair.of("V1", PolarisEntitySubType.ICEBERG_VIEW)));
     // list all principals
     this.validateListReturn(
         null,
@@ -1903,8 +1903,8 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity T6v1 =
         this.ensureExistsByName(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T6");
     Assertions.assertThat(T6v1).isNotNull();
 
@@ -1940,8 +1940,8 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity T5v1 =
         this.ensureExistsByName(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T5");
     T5v1.setId(100000L);
     PolarisBaseEntity notExists =
@@ -1981,8 +1981,8 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity T6 =
         this.ensureExistsByName(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T6");
     Assertions.assertThat(T6).isNotNull();
 
@@ -2001,22 +2001,22 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity T1 =
         this.ensureExistsByName(
             List.of(catalog, N1, N1_N2),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T1");
     this.dropEntity(List.of(catalog, N1, N1_N2), T1);
     PolarisBaseEntity T2 =
         this.ensureExistsByName(
             List.of(catalog, N1, N1_N2),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T2");
     this.dropEntity(List.of(catalog, N1, N1_N2), T2);
     PolarisBaseEntity V1 =
         this.ensureExistsByName(
             List.of(catalog, N1, N1_N2),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.VIEW,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_VIEW,
             "V1");
     this.dropEntity(List.of(catalog, N1, N1_N2), V1);
     this.dropEntity(List.of(catalog, N1), N1_N2);
@@ -2026,15 +2026,15 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity T3 =
         this.ensureExistsByName(
             List.of(catalog, N1, N1_N3),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T3");
     this.dropEntity(List.of(catalog, N1, N1_N3), T3);
     PolarisBaseEntity V2 =
         this.ensureExistsByName(
             List.of(catalog, N1, N1_N3),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.VIEW,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_VIEW,
             "V2");
     this.dropEntity(List.of(catalog, N1, N1_N3), V2);
     this.dropEntity(List.of(catalog, N1), N1_N3);
@@ -2042,8 +2042,8 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity T4 =
         this.ensureExistsByName(
             List.of(catalog, N1),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T4");
     this.dropEntity(List.of(catalog, N1), T4);
     this.dropEntity(List.of(catalog), N1);
@@ -2051,8 +2051,8 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity T5 =
         this.ensureExistsByName(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
-            PolarisEntitySubType.TABLE,
+            PolarisEntityType.TABLE_LIKE,
+            PolarisEntitySubType.ICEBERG_TABLE,
             "T5");
     this.dropEntity(List.of(catalog, N5, N5_N6), T5);
     this.dropEntity(List.of(catalog, N5), N5_N6);
@@ -2171,7 +2171,7 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity N5_N6_T5 =
         this.ensureExistsByName(
             List.of(catalog, N5, N5_N6),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
+            PolarisEntityType.TABLE_LIKE,
             PolarisEntitySubType.ANY_SUBTYPE,
             "T5");
 
@@ -2341,7 +2341,7 @@ public class PolarisTestMetaStoreManager {
     PolarisBaseEntity N1_N2_T1 =
         this.ensureExistsByName(
             List.of(catalog, N1, N1_N2),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
+            PolarisEntityType.TABLE_LIKE,
             PolarisEntitySubType.ANY_SUBTYPE,
             "T1");
     // view with the same name exists, should fail
@@ -2436,15 +2436,11 @@ public class PolarisTestMetaStoreManager {
 
     // now validate that load something which does not exist, will also work
     this.loadCacheEntryByName(
-        N1.getCatalogId(),
-        N1.getId(),
-        PolarisEntityType.ICEBERG_TABLE_LIKE,
-        "do_not_exists",
-        false);
+        N1.getCatalogId(), N1.getId(), PolarisEntityType.TABLE_LIKE, 
"do_not_exists", false);
     this.loadCacheEntryById(N1.getCatalogId() + 1000, N1.getId(), 
N1.getType(), false);
 
     // refresh a purged entity
     this.refreshCacheEntry(
-        1, 1, PolarisEntityType.ICEBERG_TABLE_LIKE, N1.getCatalogId() + 1000, 
N1.getId(), false);
+        1, 1, PolarisEntityType.TABLE_LIKE, N1.getCatalogId() + 1000, 
N1.getId(), false);
   }
 }
diff --git 
a/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/GenericTableCatalogTest.java
 
b/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/GenericTableCatalogTest.java
index 57d49226c..9fcdabc24 100644
--- 
a/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/GenericTableCatalogTest.java
+++ 
b/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/GenericTableCatalogTest.java
@@ -18,12 +18,15 @@
  */
 package org.apache.polaris.service.quarkus.catalog;
 
+import static org.apache.iceberg.types.Types.NestedField.required;
 import static org.mockito.ArgumentMatchers.isA;
 import static org.mockito.Mockito.when;
 
+import com.google.common.collect.ImmutableMap;
 import io.quarkus.test.junit.QuarkusMock;
 import io.quarkus.test.junit.QuarkusTest;
 import io.quarkus.test.junit.QuarkusTestProfile;
+import io.quarkus.test.junit.TestProfile;
 import jakarta.inject.Inject;
 import jakarta.ws.rs.core.SecurityContext;
 import java.io.IOException;
@@ -34,8 +37,11 @@ import java.util.Map;
 import java.util.Set;
 import java.util.function.Supplier;
 import org.apache.commons.lang3.NotImplementedException;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Types;
 import org.apache.polaris.core.PolarisCallContext;
 import org.apache.polaris.core.PolarisDiagnostics;
 import org.apache.polaris.core.admin.model.AwsStorageConfigInfo;
@@ -47,12 +53,11 @@ import 
org.apache.polaris.core.config.PolarisConfigurationStore;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.context.RealmContext;
 import org.apache.polaris.core.entity.CatalogEntity;
-import org.apache.polaris.core.entity.GenericTableEntity;
 import org.apache.polaris.core.entity.PolarisEntity;
 import org.apache.polaris.core.entity.PolarisEntitySubType;
 import org.apache.polaris.core.entity.PolarisEntityType;
 import org.apache.polaris.core.entity.PrincipalEntity;
-import org.apache.polaris.core.persistence.BasePersistence;
+import org.apache.polaris.core.entity.table.GenericTableEntity;
 import org.apache.polaris.core.persistence.MetaStoreManagerFactory;
 import org.apache.polaris.core.persistence.PolarisEntityManager;
 import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
@@ -60,6 +65,7 @@ import 
org.apache.polaris.core.persistence.bootstrap.RootCredentialsSet;
 import org.apache.polaris.core.persistence.cache.EntityCache;
 import org.apache.polaris.core.persistence.dao.entity.BaseResult;
 import org.apache.polaris.core.persistence.dao.entity.PrincipalSecretsResult;
+import 
org.apache.polaris.core.persistence.transactional.TransactionalPersistence;
 import org.apache.polaris.core.storage.PolarisStorageIntegration;
 import org.apache.polaris.core.storage.PolarisStorageIntegrationProvider;
 import org.apache.polaris.core.storage.aws.AwsCredentialsStorageIntegration;
@@ -69,7 +75,9 @@ import org.apache.polaris.service.admin.PolarisAdminService;
 import org.apache.polaris.service.catalog.PolarisPassthroughResolutionView;
 import org.apache.polaris.service.catalog.generic.GenericTableCatalog;
 import org.apache.polaris.service.catalog.iceberg.IcebergCatalog;
+import org.apache.polaris.service.catalog.io.DefaultFileIOFactory;
 import org.apache.polaris.service.catalog.io.FileIOFactory;
+import org.apache.polaris.service.config.RealmEntityManagerFactory;
 import 
org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl;
 import org.apache.polaris.service.task.TaskExecutor;
 import org.assertj.core.api.Assertions;
@@ -85,12 +93,20 @@ import 
software.amazon.awssdk.services.sts.model.AssumeRoleResponse;
 import software.amazon.awssdk.services.sts.model.Credentials;
 
 @QuarkusTest
+@TestProfile(GenericTableCatalogTest.Profile.class)
 public class GenericTableCatalogTest {
+
   public static class Profile implements QuarkusTestProfile {
 
     @Override
     public Map<String, String> getConfigOverrides() {
-      return Map.of();
+      return Map.of(
+          "polaris.features.defaults.\"ALLOW_SPECIFYING_FILE_IO_IMPL\"",
+          "true",
+          
"polaris.features.defaults.\"INITIALIZE_DEFAULT_CATALOG_FILEIO_FOR_TEST\"",
+          "true",
+          "polaris.features.defaults.\"SUPPORTED_CATALOG_STORAGE_TYPES\"",
+          "[\"FILE\"]");
     }
   }
 
@@ -120,6 +136,11 @@ public class GenericTableCatalogTest {
   private PolarisEntity catalogEntity;
   private SecurityContext securityContext;
 
+  protected static final Schema SCHEMA =
+      new Schema(
+          required(3, "id", Types.IntegerType.get(), "unique ID 🤪"),
+          required(4, "data", Types.StringType.get()));
+
   @BeforeAll
   public static void setUpMocks() {
     PolarisStorageIntegrationProviderImpl mock =
@@ -199,6 +220,10 @@ public class GenericTableCatalogTest {
         new PolarisPassthroughResolutionView(
             callContext, entityManager, securityContext, CATALOG_NAME);
     TaskExecutor taskExecutor = Mockito.mock();
+    RealmEntityManagerFactory realmEntityManagerFactory =
+        new RealmEntityManagerFactory(createMockMetaStoreManagerFactory());
+    this.fileIOFactory =
+        new DefaultFileIOFactory(realmEntityManagerFactory, managerFactory, 
configurationStore);
 
     StsClient stsClient = Mockito.mock(StsClient.class);
     when(stsClient.assumeRole(isA(AssumeRoleRequest.class)))
@@ -235,6 +260,10 @@ public class GenericTableCatalogTest {
             securityContext,
             taskExecutor,
             fileIOFactory);
+    this.icebergCatalog.initialize(
+        CATALOG_NAME,
+        ImmutableMap.of(
+            CatalogProperties.FILE_IO_IMPL, 
"org.apache.iceberg.inmemory.InMemoryFileIO"));
   }
 
   @AfterEach
@@ -250,8 +279,9 @@ public class GenericTableCatalogTest {
       }
 
       @Override
-      public Supplier<BasePersistence> getOrCreateSessionSupplier(RealmContext 
realmContext) {
-        return () -> polarisContext.getMetaStore();
+      public Supplier<TransactionalPersistence> getOrCreateSessionSupplier(
+          RealmContext realmContext) {
+        return () -> ((TransactionalPersistence) 
polarisContext.getMetaStore());
       }
 
       @Override
@@ -288,6 +318,40 @@ public class GenericTableCatalogTest {
         .doesNotThrowAnyException();
   }
 
+  @Test
+  public void testGenericTableAlreadyExists() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+    genericTableCatalog.createGenericTable(TableIdentifier.of("ns", "t1"), 
"format1", Map.of());
+
+    Assertions.assertThatCode(
+            () ->
+                genericTableCatalog.createGenericTable(
+                    TableIdentifier.of("ns", "t1"), "format2", Map.of()))
+        .hasMessageContaining("already exists");
+
+    Assertions.assertThatCode(
+            () -> icebergCatalog.createTable(TableIdentifier.of("ns", "t1"), 
SCHEMA))
+        .hasMessageContaining("already exists");
+  }
+
+  @Test
+  public void testIcebergTableAlreadyExists() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+    icebergCatalog.createTable(TableIdentifier.of("ns", "t1"), SCHEMA);
+
+    Assertions.assertThatCode(
+            () ->
+                genericTableCatalog.createGenericTable(
+                    TableIdentifier.of("ns", "t1"), "format2", Map.of()))
+        .hasMessageContaining("already exists");
+
+    Assertions.assertThatCode(
+            () -> icebergCatalog.createTable(TableIdentifier.of("ns", "t1"), 
SCHEMA))
+        .hasMessageContaining("already exists");
+  }
+
   @Test
   public void testGenericTableRoundTrip() {
     Namespace namespace = Namespace.of("ns");
@@ -306,4 +370,194 @@ public class GenericTableCatalogTest {
     
Assertions.assertThat(resultEntity.getPropertiesAsMap()).isEqualTo(properties);
     Assertions.assertThat(resultEntity.getName()).isEqualTo(tableName);
   }
+
+  @Test
+  public void testLoadNonExistentTable() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    Assertions.assertThatCode(
+            () -> 
genericTableCatalog.loadGenericTable(TableIdentifier.of("ns", "t1")))
+        .hasMessageContaining("does not exist: ns.t1");
+  }
+
+  @Test
+  public void testReadIcebergTableAsGeneric() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    String tableName = "t1";
+
+    icebergCatalog.createTable(TableIdentifier.of("ns", tableName), SCHEMA);
+    Assertions.assertThatCode(
+            () -> 
genericTableCatalog.loadGenericTable(TableIdentifier.of("ns", tableName)))
+        .hasMessageContaining("does not exist: ns.t1");
+  }
+
+  @Test
+  public void testReadIcebergViewAsGeneric() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    String tableName = "t1";
+
+    icebergCatalog.buildView(TableIdentifier.of("ns", tableName));
+    Assertions.assertThatCode(
+            () -> 
genericTableCatalog.loadGenericTable(TableIdentifier.of("ns", tableName)))
+        .hasMessageContaining("does not exist: ns.t1");
+  }
+
+  @Test
+  public void testReadGenericAsIcebergTable() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    String tableName = "t1";
+
+    genericTableCatalog.createGenericTable(TableIdentifier.of("ns", 
tableName), "format", Map.of());
+    Assertions.assertThatCode(() -> 
icebergCatalog.loadTable(TableIdentifier.of("ns", tableName)))
+        .hasMessageContaining("does not exist: ns.t1");
+  }
+
+  @Test
+  public void testReadGenericAsIcebergView() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    String tableName = "t1";
+
+    genericTableCatalog.createGenericTable(TableIdentifier.of("ns", 
tableName), "format", Map.of());
+    Assertions.assertThatCode(() -> 
icebergCatalog.loadView(TableIdentifier.of("ns", tableName)))
+        .hasMessageContaining("does not exist: ns.t1");
+  }
+
+  @Test
+  public void testListTables() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    for (int i = 0; i < 10; i++) {
+      genericTableCatalog.createGenericTable(TableIdentifier.of("ns", "t" + 
i), "format", Map.of());
+    }
+
+    List<TableIdentifier> listResult = 
genericTableCatalog.listGenericTables(namespace);
+
+    Assertions.assertThat(listResult.size()).isEqualTo(10);
+    
Assertions.assertThat(listResult.stream().map(TableIdentifier::toString).toList())
+        
.isEqualTo(listResult.stream().map(TableIdentifier::toString).sorted().toList());
+
+    Assertions.assertThat(icebergCatalog.listTables(namespace)).isEmpty();
+  }
+
+  @Test
+  public void testListTablesEmpty() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    for (int i = 0; i < 10; i++) {
+      icebergCatalog.createTable(TableIdentifier.of("ns", "t" + i), SCHEMA);
+    }
+
+    
Assertions.assertThat(icebergCatalog.listTables(namespace).size()).isEqualTo(10);
+    
Assertions.assertThat(genericTableCatalog.listGenericTables(namespace)).isEmpty();
+  }
+
+  @Test
+  public void testListTablesNoNamespace() {
+    Namespace namespace = Namespace.of("ns");
+
+    Assertions.assertThatCode(() -> 
genericTableCatalog.listGenericTables(namespace))
+        .hasMessageContaining("Namespace");
+  }
+
+  @Test
+  public void testListIcebergTables() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    for (int i = 0; i < 10; i++) {
+      icebergCatalog.createTable(TableIdentifier.of("ns", "t" + i), SCHEMA);
+    }
+
+    List<TableIdentifier> listResult = icebergCatalog.listTables(namespace);
+
+    Assertions.assertThat(listResult.size()).isEqualTo(10);
+    
Assertions.assertThat(listResult.stream().map(TableIdentifier::toString).toList())
+        
.isEqualTo(listResult.stream().map(TableIdentifier::toString).sorted().toList());
+
+    
Assertions.assertThat(genericTableCatalog.listGenericTables(namespace)).isEmpty();
+  }
+
+  @Test
+  public void testListMixedTables() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    for (int i = 0; i < 10; i++) {
+      icebergCatalog.createTable(TableIdentifier.of("ns", "i" + i), SCHEMA);
+    }
+
+    for (int i = 0; i < 10; i++) {
+      genericTableCatalog.createGenericTable(TableIdentifier.of("ns", "g" + 
i), "format", Map.of());
+    }
+
+    
Assertions.assertThat(genericTableCatalog.listGenericTables(namespace).size()).isEqualTo(10);
+    
Assertions.assertThat(icebergCatalog.listTables(namespace).size()).isEqualTo(10);
+  }
+
+  @Test
+  public void testDropNonExistentTable() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    Assertions.assertThatCode(
+            () -> 
genericTableCatalog.dropGenericTable(TableIdentifier.of("ns", "t1")))
+        .hasMessageContaining("Generic table does not exist: ns.t1");
+  }
+
+  @Test
+  public void testDropNonExistentNamespace() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+
+    Assertions.assertThatCode(
+            () -> 
genericTableCatalog.dropGenericTable(TableIdentifier.of("ns2", "t1")))
+        .hasMessageContaining("Generic table does not exist: ns2.t1");
+  }
+
+  @Test
+  public void testDropIcebergTable() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+    icebergCatalog.createTable(TableIdentifier.of("ns", "t1"), SCHEMA);
+
+    Assertions.assertThatCode(
+            () -> 
genericTableCatalog.dropGenericTable(TableIdentifier.of("ns", "t1")))
+        .hasMessageContaining("Generic table does not exist: ns.t1");
+
+    Assertions.assertThatCode(() -> 
icebergCatalog.dropTable(TableIdentifier.of("ns", "t1")))
+        .doesNotThrowAnyException();
+  }
+
+  @Test
+  public void testDropViaIceberg() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+    genericTableCatalog.createGenericTable(TableIdentifier.of("ns", "t1"), 
"format", Map.of());
+
+    Assertions.assertThat(icebergCatalog.dropTable(TableIdentifier.of("ns", 
"t1"))).isFalse();
+    
Assertions.assertThat(genericTableCatalog.loadGenericTable(TableIdentifier.of("ns",
 "t1")))
+        .isNotNull();
+  }
+
+  @Test
+  public void testDropViaIcebergView() {
+    Namespace namespace = Namespace.of("ns");
+    icebergCatalog.createNamespace(namespace);
+    genericTableCatalog.createGenericTable(TableIdentifier.of("ns", "t1"), 
"format", Map.of());
+
+    Assertions.assertThat(icebergCatalog.dropView(TableIdentifier.of("ns", 
"t1"))).isFalse();
+    
Assertions.assertThat(genericTableCatalog.loadGenericTable(TableIdentifier.of("ns",
 "t1")))
+        .isNotNull();
+  }
 }
diff --git 
a/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java
 
b/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java
index 3dafaf109..8d520b9a9 100644
--- 
a/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java
+++ 
b/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/catalog/IcebergCatalogTest.java
@@ -1710,7 +1710,7 @@ public abstract class IcebergCatalogTest extends 
CatalogTests<IcebergCatalog> {
     doReturn(
             new EntityResult(
                 BaseResult.ReturnStatus.ENTITY_ALREADY_EXISTS,
-                PolarisEntitySubType.TABLE.getCode()))
+                PolarisEntitySubType.ICEBERG_TABLE.getCode()))
         .when(spyMetaStore)
         .createEntityIfNotExists(any(), any(), any());
     Assertions.assertThatThrownBy(() -> catalog.createTable(table, SCHEMA))
diff --git 
a/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java
 
b/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java
index d723f6caf..33e96e2a1 100644
--- 
a/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java
+++ 
b/quarkus/service/src/test/java/org/apache/polaris/service/quarkus/task/TableCleanupTaskHandlerTest.java
@@ -43,11 +43,11 @@ import org.apache.polaris.core.PolarisCallContext;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.context.RealmContext;
 import org.apache.polaris.core.entity.AsyncTaskType;
-import org.apache.polaris.core.entity.IcebergTableLikeEntity;
 import org.apache.polaris.core.entity.PolarisBaseEntity;
 import org.apache.polaris.core.entity.PolarisEntityType;
 import org.apache.polaris.core.entity.PolarisTaskConstants;
 import org.apache.polaris.core.entity.TaskEntity;
+import org.apache.polaris.core.entity.table.IcebergTableLikeEntity;
 import org.apache.polaris.core.persistence.BasePersistence;
 import org.apache.polaris.core.persistence.MetaStoreManagerFactory;
 import org.apache.polaris.core.persistence.PolarisResolvedPathWrapper;
diff --git 
a/service/common/src/main/java/org/apache/polaris/service/admin/PolarisAdminService.java
 
b/service/common/src/main/java/org/apache/polaris/service/admin/PolarisAdminService.java
index bedb10165..effcededd 100644
--- 
a/service/common/src/main/java/org/apache/polaris/service/admin/PolarisAdminService.java
+++ 
b/service/common/src/main/java/org/apache/polaris/service/admin/PolarisAdminService.java
@@ -69,7 +69,6 @@ import org.apache.polaris.core.config.FeatureConfiguration;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.entity.CatalogEntity;
 import org.apache.polaris.core.entity.CatalogRoleEntity;
-import org.apache.polaris.core.entity.IcebergTableLikeEntity;
 import org.apache.polaris.core.entity.NamespaceEntity;
 import org.apache.polaris.core.entity.PolarisBaseEntity;
 import org.apache.polaris.core.entity.PolarisEntity;
@@ -80,6 +79,7 @@ import org.apache.polaris.core.entity.PolarisPrincipalSecrets;
 import org.apache.polaris.core.entity.PolarisPrivilege;
 import org.apache.polaris.core.entity.PrincipalEntity;
 import org.apache.polaris.core.entity.PrincipalRoleEntity;
+import org.apache.polaris.core.entity.table.IcebergTableLikeEntity;
 import org.apache.polaris.core.persistence.PolarisEntityManager;
 import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
 import org.apache.polaris.core.persistence.PolarisResolvedPathWrapper;
@@ -461,8 +461,7 @@ public class PolarisAdminService {
         entityManager.prepareResolutionManifest(callContext, securityContext, 
catalogName);
     resolutionManifest.addPath(
         new ResolverPath(
-            PolarisCatalogHelpers.tableIdentifierToList(identifier),
-            PolarisEntityType.ICEBERG_TABLE_LIKE),
+            PolarisCatalogHelpers.tableIdentifierToList(identifier), 
PolarisEntityType.TABLE_LIKE),
         identifier);
     resolutionManifest.addPath(
         new ResolverPath(List.of(catalogRoleName), 
PolarisEntityType.CATALOG_ROLE),
@@ -472,9 +471,8 @@ public class PolarisAdminService {
     if (status.getStatus() == 
ResolverStatus.StatusEnum.ENTITY_COULD_NOT_BE_RESOLVED) {
       throw new NotFoundException("Catalog not found: %s", catalogName);
     } else if (status.getStatus() == 
ResolverStatus.StatusEnum.PATH_COULD_NOT_BE_FULLY_RESOLVED) {
-      if (status.getFailedToResolvePath().getLastEntityType()
-          == PolarisEntityType.ICEBERG_TABLE_LIKE) {
-        if (subType == PolarisEntitySubType.TABLE) {
+      if (status.getFailedToResolvePath().getLastEntityType() == 
PolarisEntityType.TABLE_LIKE) {
+        if (subType == PolarisEntitySubType.ICEBERG_TABLE) {
           throw new NoSuchTableException("Table does not exist: %s", 
identifier);
         } else {
           throw new NoSuchViewException("View does not exist: %s", identifier);
@@ -485,8 +483,7 @@ public class PolarisAdminService {
     }
 
     PolarisResolvedPathWrapper tableLikeWrapper =
-        resolutionManifest.getResolvedPath(
-            identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, subType, true);
+        resolutionManifest.getResolvedPath(identifier, 
PolarisEntityType.TABLE_LIKE, subType, true);
     PolarisResolvedPathWrapper catalogRoleWrapper =
         resolutionManifest.getResolvedPath(catalogRoleName, true);
 
@@ -1498,10 +1495,10 @@ public class PolarisAdminService {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.ADD_TABLE_GRANT_TO_CATALOG_ROLE;
 
     authorizeGrantOnTableLikeOperationOrThrow(
-        op, catalogName, PolarisEntitySubType.TABLE, identifier, 
catalogRoleName);
+        op, catalogName, PolarisEntitySubType.ICEBERG_TABLE, identifier, 
catalogRoleName);
 
     return grantPrivilegeOnTableLikeToRole(
-        catalogName, catalogRoleName, identifier, PolarisEntitySubType.TABLE, 
privilege);
+        catalogName, catalogRoleName, identifier, 
PolarisEntitySubType.ICEBERG_TABLE, privilege);
   }
 
   public boolean revokePrivilegeOnTableFromRole(
@@ -1513,10 +1510,10 @@ public class PolarisAdminService {
         PolarisAuthorizableOperation.REVOKE_TABLE_GRANT_FROM_CATALOG_ROLE;
 
     authorizeGrantOnTableLikeOperationOrThrow(
-        op, catalogName, PolarisEntitySubType.TABLE, identifier, 
catalogRoleName);
+        op, catalogName, PolarisEntitySubType.ICEBERG_TABLE, identifier, 
catalogRoleName);
 
     return revokePrivilegeOnTableLikeFromRole(
-        catalogName, catalogRoleName, identifier, PolarisEntitySubType.TABLE, 
privilege);
+        catalogName, catalogRoleName, identifier, 
PolarisEntitySubType.ICEBERG_TABLE, privilege);
   }
 
   public boolean grantPrivilegeOnViewToRole(
@@ -1527,10 +1524,10 @@ public class PolarisAdminService {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.ADD_VIEW_GRANT_TO_CATALOG_ROLE;
 
     authorizeGrantOnTableLikeOperationOrThrow(
-        op, catalogName, PolarisEntitySubType.VIEW, identifier, 
catalogRoleName);
+        op, catalogName, PolarisEntitySubType.ICEBERG_VIEW, identifier, 
catalogRoleName);
 
     return grantPrivilegeOnTableLikeToRole(
-        catalogName, catalogRoleName, identifier, PolarisEntitySubType.VIEW, 
privilege);
+        catalogName, catalogRoleName, identifier, 
PolarisEntitySubType.ICEBERG_VIEW, privilege);
   }
 
   public boolean revokePrivilegeOnViewFromRole(
@@ -1542,10 +1539,10 @@ public class PolarisAdminService {
         PolarisAuthorizableOperation.REVOKE_VIEW_GRANT_FROM_CATALOG_ROLE;
 
     authorizeGrantOnTableLikeOperationOrThrow(
-        op, catalogName, PolarisEntitySubType.VIEW, identifier, 
catalogRoleName);
+        op, catalogName, PolarisEntitySubType.ICEBERG_VIEW, identifier, 
catalogRoleName);
 
     return revokePrivilegeOnTableLikeFromRole(
-        catalogName, catalogRoleName, identifier, PolarisEntitySubType.VIEW, 
privilege);
+        catalogName, catalogRoleName, identifier, 
PolarisEntitySubType.ICEBERG_VIEW, privilege);
   }
 
   public List<PolarisEntity> listAssigneePrincipalRolesForCatalogRole(
@@ -1607,9 +1604,9 @@ public class PolarisAdminService {
               namespaceGrants.add(grant);
               break;
             }
-          case ICEBERG_TABLE_LIKE:
+          case TABLE_LIKE:
             {
-              if (baseEntity.getSubType() == PolarisEntitySubType.TABLE) {
+              if (baseEntity.getSubType() == 
PolarisEntitySubType.ICEBERG_TABLE) {
                 TableIdentifier identifier =
                     IcebergTableLikeEntity.of(baseEntity).getTableIdentifier();
                 TableGrant grant =
@@ -1707,10 +1704,9 @@ public class PolarisAdminService {
             .orElseThrow(() -> new NotFoundException("CatalogRole %s not 
found", catalogRoleName));
 
     PolarisResolvedPathWrapper resolvedPathWrapper =
-        resolutionManifest.getResolvedPath(
-            identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, subType);
+        resolutionManifest.getResolvedPath(identifier, 
PolarisEntityType.TABLE_LIKE, subType);
     if (resolvedPathWrapper == null) {
-      if (subType == PolarisEntitySubType.VIEW) {
+      if (subType == PolarisEntitySubType.ICEBERG_VIEW) {
         throw new NotFoundException("View %s not found", identifier);
       } else {
         throw new NotFoundException("Table %s not found", identifier);
@@ -1746,10 +1742,9 @@ public class PolarisAdminService {
             .orElseThrow(() -> new NotFoundException("CatalogRole %s not 
found", catalogRoleName));
 
     PolarisResolvedPathWrapper resolvedPathWrapper =
-        resolutionManifest.getResolvedPath(
-            identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, subType);
+        resolutionManifest.getResolvedPath(identifier, 
PolarisEntityType.TABLE_LIKE, subType);
     if (resolvedPathWrapper == null) {
-      if (subType == PolarisEntitySubType.VIEW) {
+      if (subType == PolarisEntitySubType.ICEBERG_VIEW) {
         throw new NotFoundException("View %s not found", identifier);
       } else {
         throw new NotFoundException("Table %s not found", identifier);
diff --git 
a/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalog.java
 
b/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalog.java
index 86c4f0bbf..f686f08e9 100644
--- 
a/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalog.java
+++ 
b/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalog.java
@@ -21,19 +21,23 @@ package org.apache.polaris.service.catalog.generic;
 import jakarta.ws.rs.core.SecurityContext;
 import java.util.List;
 import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.polaris.core.catalog.PolarisCatalogHelpers;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.entity.CatalogEntity;
-import org.apache.polaris.core.entity.GenericTableEntity;
 import org.apache.polaris.core.entity.PolarisEntity;
 import org.apache.polaris.core.entity.PolarisEntitySubType;
 import org.apache.polaris.core.entity.PolarisEntityType;
+import org.apache.polaris.core.entity.table.GenericTableEntity;
 import org.apache.polaris.core.persistence.PolarisEntityManager;
 import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
 import org.apache.polaris.core.persistence.PolarisResolvedPathWrapper;
 import org.apache.polaris.core.persistence.dao.entity.BaseResult;
+import org.apache.polaris.core.persistence.dao.entity.DropEntityResult;
 import org.apache.polaris.core.persistence.dao.entity.EntityResult;
 import 
org.apache.polaris.core.persistence.resolver.PolarisResolutionManifestCatalogView;
 import org.apache.polaris.service.catalog.io.FileIOFactory;
@@ -97,10 +101,9 @@ public class GenericTableCatalog {
 
     List<PolarisEntity> catalogPath = resolvedParent.getRawFullPath();
 
-    // TODO we need to filter by type here?
     PolarisResolvedPathWrapper resolvedEntities =
         resolvedEntityView.getPassthroughResolvedPath(
-            tableIdentifier, PolarisEntityType.GENERIC_TABLE, 
PolarisEntitySubType.ANY_SUBTYPE);
+            tableIdentifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ANY_SUBTYPE);
     GenericTableEntity entity =
         GenericTableEntity.of(
             resolvedEntities == null ? null : 
resolvedEntities.getRawLeafEntity());
@@ -146,17 +149,58 @@ public class GenericTableCatalog {
   }
 
   public GenericTableEntity loadGenericTable(TableIdentifier tableIdentifier) {
-    // TODO we need to filter by type here?
     PolarisResolvedPathWrapper resolvedEntities =
         resolvedEntityView.getPassthroughResolvedPath(
-            tableIdentifier, PolarisEntityType.GENERIC_TABLE, 
PolarisEntitySubType.ANY_SUBTYPE);
+            tableIdentifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.GENERIC_TABLE);
     GenericTableEntity entity =
         GenericTableEntity.of(
             resolvedEntities == null ? null : 
resolvedEntities.getRawLeafEntity());
     if (null == entity) {
-      throw new NoSuchTableException("Table does not exist: %s", 
tableIdentifier);
+      throw new NoSuchTableException("Generic table does not exist: %s", 
tableIdentifier);
     } else {
       return entity;
     }
   }
+
+  public boolean dropGenericTable(TableIdentifier tableIdentifier) {
+    PolarisResolvedPathWrapper resolvedEntities =
+        resolvedEntityView.getPassthroughResolvedPath(
+            tableIdentifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.GENERIC_TABLE);
+
+    if (resolvedEntities == null) {
+      throw new NoSuchTableException("Generic table does not exist: %s", 
tableIdentifier);
+    }
+
+    List<PolarisEntity> catalogPath = resolvedEntities.getRawParentPath();
+    PolarisEntity leafEntity = resolvedEntities.getRawLeafEntity();
+
+    DropEntityResult dropEntityResult =
+        this.metaStoreManager.dropEntityIfExists(
+            this.callContext.getPolarisCallContext(),
+            PolarisEntity.toCoreList(catalogPath),
+            leafEntity,
+            Map.of(),
+            false);
+
+    return dropEntityResult.isSuccess();
+  }
+
+  public List<TableIdentifier> listGenericTables(Namespace namespace) {
+    PolarisResolvedPathWrapper resolvedEntities = 
resolvedEntityView.getResolvedPath(namespace);
+    if (resolvedEntities == null) {
+      throw new NoSuchNamespaceException("Namespace '%s' does not exist", 
namespace);
+    }
+
+    List<PolarisEntity> catalogPath = resolvedEntities.getRawFullPath();
+    List<PolarisEntity.NameAndId> entities =
+        PolarisEntity.toNameAndIdList(
+            this.metaStoreManager
+                .listEntities(
+                    this.callContext.getPolarisCallContext(),
+                    PolarisEntity.toCoreList(catalogPath),
+                    PolarisEntityType.TABLE_LIKE,
+                    PolarisEntitySubType.GENERIC_TABLE)
+                .getEntities());
+    return PolarisCatalogHelpers.nameAndIdToTableIdentifiers(catalogPath, 
entities);
+  }
 }
diff --git 
a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java
 
b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java
index 00469cc49..ef6c99212 100644
--- 
a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java
+++ 
b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalog.java
@@ -83,13 +83,13 @@ import 
org.apache.polaris.core.config.BehaviorChangeConfiguration;
 import org.apache.polaris.core.config.FeatureConfiguration;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.entity.CatalogEntity;
-import org.apache.polaris.core.entity.IcebergTableLikeEntity;
 import org.apache.polaris.core.entity.NamespaceEntity;
 import org.apache.polaris.core.entity.PolarisEntity;
 import org.apache.polaris.core.entity.PolarisEntityConstants;
 import org.apache.polaris.core.entity.PolarisEntitySubType;
 import org.apache.polaris.core.entity.PolarisEntityType;
 import org.apache.polaris.core.entity.PolarisTaskConstants;
+import org.apache.polaris.core.entity.table.IcebergTableLikeEntity;
 import org.apache.polaris.core.persistence.PolarisEntityManager;
 import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
 import org.apache.polaris.core.persistence.PolarisResolvedPathWrapper;
@@ -440,7 +440,8 @@ public class IcebergCatalog extends BaseMetastoreViewCatalog
                 })
             .orElse(Map.of());
     DropEntityResult dropEntityResult =
-        dropTableLike(PolarisEntitySubType.TABLE, tableIdentifier, 
storageProperties, purge);
+        dropTableLike(
+            PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier, 
storageProperties, purge);
     if (!dropEntityResult.isSuccess()) {
       return false;
     }
@@ -463,7 +464,7 @@ public class IcebergCatalog extends BaseMetastoreViewCatalog
           "Cannot list tables for namespace. Namespace does not exist: %s", 
namespace);
     }
 
-    return listTableLike(PolarisEntitySubType.TABLE, namespace);
+    return listTableLike(PolarisEntitySubType.ICEBERG_TABLE, namespace);
   }
 
   @Override
@@ -472,7 +473,7 @@ public class IcebergCatalog extends BaseMetastoreViewCatalog
       return;
     }
 
-    renameTableLike(PolarisEntitySubType.TABLE, from, to);
+    renameTableLike(PolarisEntitySubType.ICEBERG_TABLE, from, to);
   }
 
   @Override
@@ -799,7 +800,7 @@ public class IcebergCatalog extends BaseMetastoreViewCatalog
           "Cannot list views for namespace. Namespace does not exist: %s", 
namespace);
     }
 
-    return listTableLike(PolarisEntitySubType.VIEW, namespace);
+    return listTableLike(PolarisEntitySubType.ICEBERG_VIEW, namespace);
   }
 
   @Override
@@ -809,7 +810,7 @@ public class IcebergCatalog extends BaseMetastoreViewCatalog
 
   @Override
   public boolean dropView(TableIdentifier identifier) {
-    return dropTableLike(PolarisEntitySubType.VIEW, identifier, Map.of(), 
true).isSuccess();
+    return dropTableLike(PolarisEntitySubType.ICEBERG_VIEW, identifier, 
Map.of(), true).isSuccess();
   }
 
   @Override
@@ -818,14 +819,14 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
       return;
     }
 
-    renameTableLike(PolarisEntitySubType.VIEW, from, to);
+    renameTableLike(PolarisEntitySubType.ICEBERG_VIEW, from, to);
   }
 
   @Override
   public boolean sendNotification(
       TableIdentifier identifier, NotificationRequest notificationRequest) {
     return sendNotificationForTableLike(
-        PolarisEntitySubType.TABLE, identifier, notificationRequest);
+        PolarisEntitySubType.ICEBERG_TABLE, identifier, notificationRequest);
   }
 
   @Override
@@ -878,7 +879,7 @@ public class IcebergCatalog extends BaseMetastoreViewCatalog
   private @Nonnull Optional<PolarisEntity> findStorageInfo(TableIdentifier 
tableIdentifier) {
     PolarisResolvedPathWrapper resolvedTableEntities =
         resolvedEntityView.getResolvedPath(
-            tableIdentifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE);
+            tableIdentifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE);
 
     PolarisResolvedPathWrapper resolvedStorageEntity =
         resolvedTableEntities == null
@@ -895,7 +896,7 @@ public class IcebergCatalog extends BaseMetastoreViewCatalog
   private void validateLocationForTableLike(TableIdentifier identifier, String 
location) {
     PolarisResolvedPathWrapper resolvedStorageEntity =
         resolvedEntityView.getResolvedPath(
-            identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.ANY_SUBTYPE);
+            identifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ANY_SUBTYPE);
     if (resolvedStorageEntity == null) {
       resolvedStorageEntity = 
resolvedEntityView.getResolvedPath(identifier.namespace());
     }
@@ -1023,7 +1024,8 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
             catalog,
             FeatureConfiguration.ALLOW_TABLE_LOCATION_OVERLAP)) {
       LOGGER.debug("Skipping location overlap validation for identifier '{}'", 
identifier);
-    } else if (validateViewOverlap || 
entity.getSubType().equals(PolarisEntitySubType.TABLE)) {
+    } else if (validateViewOverlap
+        || entity.getSubType().equals(PolarisEntitySubType.ICEBERG_TABLE)) {
       LOGGER.debug("Validating no overlap with sibling tables or namespaces");
       validateNoLocationOverlap(location, resolvedNamespace, 
identifier.name());
     }
@@ -1068,7 +1070,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
                               parentPath.stream()
                                   .map(PolarisEntity::toCore)
                                   .collect(Collectors.toList()),
-                              PolarisEntityType.ICEBERG_TABLE_LIKE,
+                              PolarisEntityType.TABLE_LIKE,
                               PolarisEntitySubType.ANY_SUBTYPE);
                   if (!siblingTablesResult.isSuccess()) {
                     throw new IllegalStateException(
@@ -1103,8 +1105,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
         tbl ->
             resolutionManifest.addPath(
                 new ResolverPath(
-                    PolarisCatalogHelpers.tableIdentifierToList(tbl),
-                    PolarisEntityType.ICEBERG_TABLE_LIKE),
+                    PolarisCatalogHelpers.tableIdentifierToList(tbl), 
PolarisEntityType.TABLE_LIKE),
                 tbl));
     siblingNamespaces.forEach(
         ns ->
@@ -1196,7 +1197,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
       // table entity instead of the statically-resolved authz resolution set.
       PolarisResolvedPathWrapper resolvedEntities =
           resolvedEntityView.getPassthroughResolvedPath(
-              tableIdentifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE);
+              tableIdentifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE);
       IcebergTableLikeEntity entity = null;
 
       if (resolvedEntities != null) {
@@ -1250,7 +1251,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
 
       PolarisResolvedPathWrapper resolvedTableEntities =
           resolvedEntityView.getPassthroughResolvedPath(
-              tableIdentifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE);
+              tableIdentifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE);
 
       // Fetch credentials for the resolved entity. The entity could be the 
table itself (if it has
       // already been stored and credentials have been configured directly) or 
it could be the
@@ -1321,32 +1322,34 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
       String newLocation = writeNewMetadataIfRequired(base == null, metadata);
       String oldLocation = base == null ? null : base.metadataFileLocation();
 
-      PolarisResolvedPathWrapper resolvedView =
-          resolvedEntityView.getPassthroughResolvedPath(
-              tableIdentifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW);
-      if (resolvedView != null) {
-        throw new AlreadyExistsException("View with same name already exists: 
%s", tableIdentifier);
-      }
-
       // TODO: Consider using the entity from doRefresh() directly to do the 
conflict detection
       // instead of a two-layer CAS (checking metadataLocation to detect 
concurrent modification
       // between doRefresh() and doCommit(), and then 
updateEntityPropertiesIfNotChanged to detect
       // concurrent
       // modification between our checking of unchanged metadataLocation here 
and actual
       // persistence-layer commit).
-      PolarisResolvedPathWrapper resolvedEntities =
+      PolarisResolvedPathWrapper resolvedPath =
           resolvedEntityView.getPassthroughResolvedPath(
-              tableIdentifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE);
+              tableIdentifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ANY_SUBTYPE);
+      if (resolvedPath != null && resolvedPath.getRawLeafEntity() != null) {
+        if (resolvedPath.getRawLeafEntity().getSubType() == 
PolarisEntitySubType.ICEBERG_VIEW) {
+          throw new AlreadyExistsException(
+              "View with same name already exists: %s", tableIdentifier);
+        } else if (resolvedPath.getRawLeafEntity().getSubType()
+            == PolarisEntitySubType.GENERIC_TABLE) {
+          throw new AlreadyExistsException(
+              "Generic table with same name already exists: %s", 
tableIdentifier);
+        }
+      }
       IcebergTableLikeEntity entity =
-          IcebergTableLikeEntity.of(
-              resolvedEntities == null ? null : 
resolvedEntities.getRawLeafEntity());
+          IcebergTableLikeEntity.of(resolvedPath == null ? null : 
resolvedPath.getRawLeafEntity());
       String existingLocation;
       if (null == entity) {
         existingLocation = null;
         entity =
             new IcebergTableLikeEntity.Builder(tableIdentifier, newLocation)
                 .setCatalogId(getCatalogId())
-                .setSubType(PolarisEntitySubType.TABLE)
+                .setSubType(PolarisEntitySubType.ICEBERG_TABLE)
                 .setBaseLocation(metadata.location())
                 .setId(
                     
getMetaStoreManager().generateNewEntityId(getCurrentPolarisContext()).getId())
@@ -1434,7 +1437,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
     public void doRefresh() {
       PolarisResolvedPathWrapper resolvedEntities =
           resolvedEntityView.getPassthroughResolvedPath(
-              identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW);
+              identifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_VIEW);
       IcebergTableLikeEntity entity = null;
 
       if (resolvedEntities != null) {
@@ -1489,14 +1492,14 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
 
       PolarisResolvedPathWrapper resolvedTable =
           resolvedEntityView.getPassthroughResolvedPath(
-              identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.TABLE);
+              identifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_TABLE);
       if (resolvedTable != null) {
         throw new AlreadyExistsException("Table with same name already exists: 
%s", identifier);
       }
 
       PolarisResolvedPathWrapper resolvedEntities =
           resolvedEntityView.getPassthroughResolvedPath(
-              identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, 
PolarisEntitySubType.VIEW);
+              identifier, PolarisEntityType.TABLE_LIKE, 
PolarisEntitySubType.ICEBERG_VIEW);
 
       // Fetch credentials for the resolved entity. The entity could be the 
view itself (if it has
       // already been stored and credentials have been configured directly) or 
it could be the
@@ -1544,7 +1547,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
         entity =
             new IcebergTableLikeEntity.Builder(identifier, newLocation)
                 .setCatalogId(getCatalogId())
-                .setSubType(PolarisEntitySubType.VIEW)
+                .setSubType(PolarisEntitySubType.ICEBERG_VIEW)
                 .setId(
                     
getMetaStoreManager().generateNewEntityId(getCurrentPolarisContext()).getId())
                 .build();
@@ -1637,9 +1640,9 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
       PolarisEntitySubType subType, TableIdentifier from, TableIdentifier to) {
     LOGGER.debug("Renaming tableLike from {} to {}", from, to);
     PolarisResolvedPathWrapper resolvedEntities =
-        resolvedEntityView.getResolvedPath(from, 
PolarisEntityType.ICEBERG_TABLE_LIKE, subType);
+        resolvedEntityView.getResolvedPath(from, PolarisEntityType.TABLE_LIKE, 
subType);
     if (resolvedEntities == null) {
-      if (subType == PolarisEntitySubType.VIEW) {
+      if (subType == PolarisEntitySubType.ICEBERG_VIEW) {
         throw new NoSuchViewException("Cannot rename %s to %s. View does not 
exist", from, to);
       } else {
         throw new NoSuchTableException("Cannot rename %s to %s. Table does not 
exist", from, to);
@@ -1698,10 +1701,10 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
               // this code path is unexpected
               throw new AlreadyExistsException(
                   "Cannot rename %s to %s. Object already exists", from, to);
-            } else if (existingEntitySubType == PolarisEntitySubType.TABLE) {
+            } else if (existingEntitySubType == 
PolarisEntitySubType.ICEBERG_TABLE) {
               throw new AlreadyExistsException(
                   "Cannot rename %s to %s. Table already exists", from, to);
-            } else if (existingEntitySubType == PolarisEntitySubType.VIEW) {
+            } else if (existingEntitySubType == 
PolarisEntitySubType.ICEBERG_VIEW) {
               throw new AlreadyExistsException(
                   "Cannot rename %s to %s. View already exists", from, to);
             }
@@ -1850,8 +1853,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
       Map<String, String> storageProperties,
       boolean purge) {
     PolarisResolvedPathWrapper resolvedEntities =
-        resolvedEntityView.getResolvedPath(
-            identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, subType);
+        resolvedEntityView.getResolvedPath(identifier, 
PolarisEntityType.TABLE_LIKE, subType);
     if (resolvedEntities == null) {
       // TODO: Error?
       return new DropEntityResult(BaseResult.ReturnStatus.ENTITY_NOT_FOUND, 
null);
@@ -1896,14 +1898,15 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
         "Handling notification request {} for tableIdentifier {}", request, 
tableIdentifier);
     PolarisResolvedPathWrapper resolvedEntities =
         resolvedEntityView.getPassthroughResolvedPath(
-            tableIdentifier, PolarisEntityType.ICEBERG_TABLE_LIKE, subType);
+            tableIdentifier, PolarisEntityType.TABLE_LIKE, subType);
 
     NotificationType notificationType = request.getNotificationType();
 
     Preconditions.checkNotNull(notificationType, "Expected a valid 
notification type.");
 
     if (notificationType == NotificationType.DROP) {
-      return dropTableLike(PolarisEntitySubType.TABLE, tableIdentifier, 
Map.of(), false /* purge */)
+      return dropTableLike(
+              PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier, Map.of(), 
false /* purge */)
           .isSuccess();
     } else if (notificationType == NotificationType.VALIDATE) {
       // In this mode we don't want to make any mutations, so we won't 
auto-create non-existing
@@ -1969,7 +1972,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
         entity =
             new IcebergTableLikeEntity.Builder(tableIdentifier, newLocation)
                 .setCatalogId(getCatalogId())
-                .setSubType(PolarisEntitySubType.TABLE)
+                .setSubType(PolarisEntitySubType.ICEBERG_TABLE)
                 .setId(
                     
getMetaStoreManager().generateNewEntityId(getCurrentPolarisContext()).getId())
                 
.setLastNotificationTimestamp(request.getPayload().getTimestamp())
@@ -2061,7 +2064,7 @@ public class IcebergCatalog extends 
BaseMetastoreViewCatalog
                 .listEntities(
                     getCurrentPolarisContext(),
                     PolarisEntity.toCoreList(catalogPath),
-                    PolarisEntityType.ICEBERG_TABLE_LIKE,
+                    PolarisEntityType.TABLE_LIKE,
                     subType)
                 .getEntities());
     return PolarisCatalogHelpers.nameAndIdToTableIdentifiers(catalogPath, 
entities);
diff --git 
a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerWrapper.java
 
b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerWrapper.java
index 543ba907c..fab2453e7 100644
--- 
a/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerWrapper.java
+++ 
b/service/common/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogHandlerWrapper.java
@@ -217,7 +217,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
         resolutionManifest.addPassthroughPath(
             new ResolverPath(
                 PolarisCatalogHelpers.tableIdentifierToList(id),
-                PolarisEntityType.ICEBERG_TABLE_LIKE,
+                PolarisEntityType.TABLE_LIKE,
                 true /* optional */),
             id);
       }
@@ -289,7 +289,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
     resolutionManifest.addPassthroughPath(
         new ResolverPath(
             PolarisCatalogHelpers.tableIdentifierToList(identifier),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
+            PolarisEntityType.TABLE_LIKE,
             true /* optional */),
         identifier);
     resolutionManifest.resolveAll();
@@ -316,15 +316,14 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
     resolutionManifest.addPassthroughPath(
         new ResolverPath(
             PolarisCatalogHelpers.tableIdentifierToList(identifier),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
+            PolarisEntityType.TABLE_LIKE,
             true /* optional */),
         identifier);
     resolutionManifest.resolveAll();
     PolarisResolvedPathWrapper target =
-        resolutionManifest.getResolvedPath(
-            identifier, PolarisEntityType.ICEBERG_TABLE_LIKE, subType, true);
+        resolutionManifest.getResolvedPath(identifier, 
PolarisEntityType.TABLE_LIKE, subType, true);
     if (target == null) {
-      if (subType == PolarisEntitySubType.TABLE) {
+      if (subType == PolarisEntitySubType.ICEBERG_TABLE) {
         throw new NoSuchTableException("Table does not exist: %s", identifier);
       } else {
         throw new NoSuchViewException("View does not exist: %s", identifier);
@@ -351,7 +350,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
             resolutionManifest.addPassthroughPath(
                 new ResolverPath(
                     PolarisCatalogHelpers.tableIdentifierToList(identifier),
-                    PolarisEntityType.ICEBERG_TABLE_LIKE),
+                    PolarisEntityType.TABLE_LIKE),
                 identifier));
 
     ResolverStatus status = resolutionManifest.resolveAll();
@@ -362,7 +361,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
       TableIdentifier identifier =
           PolarisCatalogHelpers.listToTableIdentifier(
               status.getFailedToResolvePath().getEntityNames());
-      if (subType == PolarisEntitySubType.TABLE) {
+      if (subType == PolarisEntitySubType.ICEBERG_TABLE) {
         throw new NoSuchTableException("Table does not exist: %s", identifier);
       } else {
         throw new NoSuchViewException("View does not exist: %s", identifier);
@@ -375,10 +374,10 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
                 identifier ->
                     Optional.ofNullable(
                             resolutionManifest.getResolvedPath(
-                                identifier, 
PolarisEntityType.ICEBERG_TABLE_LIKE, subType, true))
+                                identifier, PolarisEntityType.TABLE_LIKE, 
subType, true))
                         .orElseThrow(
                             () ->
-                                subType == PolarisEntitySubType.TABLE
+                                subType == PolarisEntitySubType.ICEBERG_TABLE
                                     ? new NoSuchTableException(
                                         "Table does not exist: %s", identifier)
                                     : new NoSuchViewException(
@@ -404,7 +403,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
     // Add src, dstParent, and dst(optional)
     resolutionManifest.addPath(
         new ResolverPath(
-            PolarisCatalogHelpers.tableIdentifierToList(src), 
PolarisEntityType.ICEBERG_TABLE_LIKE),
+            PolarisCatalogHelpers.tableIdentifierToList(src), 
PolarisEntityType.TABLE_LIKE),
         src);
     resolutionManifest.addPath(
         new ResolverPath(Arrays.asList(dst.namespace().levels()), 
PolarisEntityType.NAMESPACE),
@@ -412,17 +411,16 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
     resolutionManifest.addPath(
         new ResolverPath(
             PolarisCatalogHelpers.tableIdentifierToList(dst),
-            PolarisEntityType.ICEBERG_TABLE_LIKE,
+            PolarisEntityType.TABLE_LIKE,
             true /* optional */),
         dst);
     ResolverStatus status = resolutionManifest.resolveAll();
     if (status.getStatus() == 
ResolverStatus.StatusEnum.PATH_COULD_NOT_BE_FULLY_RESOLVED
         && status.getFailedToResolvePath().getLastEntityType() == 
PolarisEntityType.NAMESPACE) {
       throw new NoSuchNamespaceException("Namespace does not exist: %s", 
dst.namespace());
-    } else if (resolutionManifest.getResolvedPath(
-            src, PolarisEntityType.ICEBERG_TABLE_LIKE, subType)
+    } else if (resolutionManifest.getResolvedPath(src, 
PolarisEntityType.TABLE_LIKE, subType)
         == null) {
-      if (subType == PolarisEntitySubType.TABLE) {
+      if (subType == PolarisEntitySubType.ICEBERG_TABLE) {
         throw new NoSuchTableException("Table does not exist: %s", src);
       } else {
         throw new NoSuchViewException("View does not exist: %s", src);
@@ -437,15 +435,14 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
     // TODO: Possibly modify the exception thrown depending on whether the 
caller has privileges
     // on the parent namespace.
     PolarisEntitySubType dstLeafSubType = 
resolutionManifest.getLeafSubType(dst);
-    if (dstLeafSubType == PolarisEntitySubType.TABLE) {
+    if (dstLeafSubType == PolarisEntitySubType.ICEBERG_TABLE) {
       throw new AlreadyExistsException("Cannot rename %s to %s. Table already 
exists", src, dst);
-    } else if (dstLeafSubType == PolarisEntitySubType.VIEW) {
+    } else if (dstLeafSubType == PolarisEntitySubType.ICEBERG_VIEW) {
       throw new AlreadyExistsException("Cannot rename %s to %s. View already 
exists", src, dst);
     }
 
     PolarisResolvedPathWrapper target =
-        resolutionManifest.getResolvedPath(
-            src, PolarisEntityType.ICEBERG_TABLE_LIKE, subType, true);
+        resolutionManifest.getResolvedPath(src, PolarisEntityType.TABLE_LIKE, 
subType, true);
     PolarisResolvedPathWrapper secondary =
         resolutionManifest.getResolvedPath(dst.namespace(), true);
     authorizer.authorizeOrThrow(
@@ -773,7 +770,8 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
 
   public LoadTableResponse loadTable(TableIdentifier tableIdentifier, String 
snapshots) {
     PolarisAuthorizableOperation op = PolarisAuthorizableOperation.LOAD_TABLE;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.TABLE, 
tableIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(
+        op, PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier);
 
     return CatalogHandlers.loadTable(baseCatalog, tableIdentifier);
   }
@@ -795,10 +793,12 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
     try {
       // TODO: Refactor to have a boolean-return version of the helpers so we 
can fallthrough
       // easily.
-      authorizeBasicTableLikeOperationOrThrow(write, 
PolarisEntitySubType.TABLE, tableIdentifier);
+      authorizeBasicTableLikeOperationOrThrow(
+          write, PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier);
       actionsRequested.add(PolarisStorageActions.WRITE);
     } catch (ForbiddenException e) {
-      authorizeBasicTableLikeOperationOrThrow(read, 
PolarisEntitySubType.TABLE, tableIdentifier);
+      authorizeBasicTableLikeOperationOrThrow(
+          read, PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier);
     }
 
     PolarisResolvedPathWrapper catalogPath = 
resolutionManifest.getResolvedReferenceCatalogEntity();
@@ -876,7 +876,8 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
   public LoadTableResponse updateTable(
       TableIdentifier tableIdentifier, UpdateTableRequest request) {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.UPDATE_TABLE;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.TABLE, 
tableIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(
+        op, PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier);
 
     CatalogEntity catalog =
         CatalogEntity.of(
@@ -909,14 +910,16 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
 
   public void dropTableWithoutPurge(TableIdentifier tableIdentifier) {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.DROP_TABLE_WITHOUT_PURGE;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.TABLE, 
tableIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(
+        op, PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier);
 
     CatalogHandlers.dropTable(baseCatalog, tableIdentifier);
   }
 
   public void dropTableWithPurge(TableIdentifier tableIdentifier) {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.DROP_TABLE_WITH_PURGE;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.TABLE, 
tableIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(
+        op, PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier);
 
     CatalogEntity catalog =
         CatalogEntity.of(
@@ -932,7 +935,8 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
 
   public void tableExists(TableIdentifier tableIdentifier) {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.TABLE_EXISTS;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.TABLE, 
tableIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(
+        op, PolarisEntitySubType.ICEBERG_TABLE, tableIdentifier);
 
     // TODO: Just skip CatalogHandlers for this one maybe
     CatalogHandlers.loadTable(baseCatalog, tableIdentifier);
@@ -941,7 +945,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
   public void renameTable(RenameTableRequest request) {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.RENAME_TABLE;
     authorizeRenameTableLikeOperationOrThrow(
-        op, PolarisEntitySubType.TABLE, request.source(), 
request.destination());
+        op, PolarisEntitySubType.ICEBERG_TABLE, request.source(), 
request.destination());
 
     CatalogEntity catalog =
         CatalogEntity.of(
@@ -963,7 +967,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
     // needs TABLE_WRITE_PROPERTIES.
     authorizeCollectionOfTableLikeOperationOrThrow(
         op,
-        PolarisEntitySubType.TABLE,
+        PolarisEntitySubType.ICEBERG_TABLE,
         commitTransactionRequest.tableChanges().stream()
             .map(UpdateTableRequest::identifier)
             .toList());
@@ -1087,14 +1091,14 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
 
   public LoadViewResponse loadView(TableIdentifier viewIdentifier) {
     PolarisAuthorizableOperation op = PolarisAuthorizableOperation.LOAD_VIEW;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.VIEW, 
viewIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(op, 
PolarisEntitySubType.ICEBERG_VIEW, viewIdentifier);
 
     return CatalogHandlers.loadView(viewCatalog, viewIdentifier);
   }
 
   public LoadViewResponse replaceView(TableIdentifier viewIdentifier, 
UpdateTableRequest request) {
     PolarisAuthorizableOperation op = 
PolarisAuthorizableOperation.REPLACE_VIEW;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.VIEW, 
viewIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(op, 
PolarisEntitySubType.ICEBERG_VIEW, viewIdentifier);
 
     CatalogEntity catalog =
         CatalogEntity.of(
@@ -1110,14 +1114,14 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
 
   public void dropView(TableIdentifier viewIdentifier) {
     PolarisAuthorizableOperation op = PolarisAuthorizableOperation.DROP_VIEW;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.VIEW, 
viewIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(op, 
PolarisEntitySubType.ICEBERG_VIEW, viewIdentifier);
 
     CatalogHandlers.dropView(viewCatalog, viewIdentifier);
   }
 
   public void viewExists(TableIdentifier viewIdentifier) {
     PolarisAuthorizableOperation op = PolarisAuthorizableOperation.VIEW_EXISTS;
-    authorizeBasicTableLikeOperationOrThrow(op, PolarisEntitySubType.VIEW, 
viewIdentifier);
+    authorizeBasicTableLikeOperationOrThrow(op, 
PolarisEntitySubType.ICEBERG_VIEW, viewIdentifier);
 
     // TODO: Just skip CatalogHandlers for this one maybe
     CatalogHandlers.loadView(viewCatalog, viewIdentifier);
@@ -1126,7 +1130,7 @@ public class IcebergCatalogHandlerWrapper implements 
AutoCloseable {
   public void renameView(RenameTableRequest request) {
     PolarisAuthorizableOperation op = PolarisAuthorizableOperation.RENAME_VIEW;
     authorizeRenameTableLikeOperationOrThrow(
-        op, PolarisEntitySubType.VIEW, request.source(), 
request.destination());
+        op, PolarisEntitySubType.ICEBERG_VIEW, request.source(), 
request.destination());
 
     CatalogEntity catalog =
         CatalogEntity.of(
diff --git 
a/service/common/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java
 
b/service/common/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java
index 35ba9440e..f9f1c2f35 100644
--- 
a/service/common/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java
+++ 
b/service/common/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java
@@ -33,11 +33,11 @@ import org.apache.iceberg.io.FileIO;
 import org.apache.polaris.core.PolarisCallContext;
 import org.apache.polaris.core.context.CallContext;
 import org.apache.polaris.core.entity.AsyncTaskType;
-import org.apache.polaris.core.entity.IcebergTableLikeEntity;
 import org.apache.polaris.core.entity.PolarisBaseEntity;
 import org.apache.polaris.core.entity.PolarisEntity;
 import org.apache.polaris.core.entity.PolarisEntityType;
 import org.apache.polaris.core.entity.TaskEntity;
+import org.apache.polaris.core.entity.table.IcebergTableLikeEntity;
 import org.apache.polaris.core.persistence.MetaStoreManagerFactory;
 import org.apache.polaris.core.persistence.PolarisMetaStoreManager;
 import org.slf4j.Logger;
@@ -71,7 +71,7 @@ public class TableCleanupTaskHandler implements TaskHandler {
 
   private boolean taskEntityIsTable(TaskEntity task) {
     PolarisEntity entity = 
PolarisEntity.of((task.readData(PolarisBaseEntity.class)));
-    return entity.getType().equals(PolarisEntityType.ICEBERG_TABLE_LIKE);
+    return entity.getType().equals(PolarisEntityType.TABLE_LIKE);
   }
 
   @Override
diff --git 
a/service/common/src/main/java/org/apache/polaris/service/task/TaskFileIOSupplier.java
 
b/service/common/src/main/java/org/apache/polaris/service/task/TaskFileIOSupplier.java
index b87c06271..e3a1ddd48 100644
--- 
a/service/common/src/main/java/org/apache/polaris/service/task/TaskFileIOSupplier.java
+++ 
b/service/common/src/main/java/org/apache/polaris/service/task/TaskFileIOSupplier.java
@@ -29,9 +29,9 @@ import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.io.FileIO;
 import org.apache.polaris.core.context.CallContext;
-import org.apache.polaris.core.entity.IcebergTableLikeEntity;
 import org.apache.polaris.core.entity.PolarisTaskConstants;
 import org.apache.polaris.core.entity.TaskEntity;
+import org.apache.polaris.core.entity.table.IcebergTableLikeEntity;
 import org.apache.polaris.core.persistence.PolarisResolvedPathWrapper;
 import org.apache.polaris.core.persistence.ResolvedPolarisEntity;
 import org.apache.polaris.core.storage.PolarisStorageActions;

Reply via email to