pvary commented on code in PR #8907:
URL: https://github.com/apache/iceberg/pull/8907#discussion_r1441980492


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.iceberg.hive;
+
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Objects;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergViewException;
+import org.apache.iceberg.exceptions.NoSuchViewException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.util.MetastoreOperationsUtil;
+import org.apache.iceberg.view.BaseViewOperations;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Hive implementation of Iceberg ViewOperations. */
+final class HiveViewOperations extends BaseViewOperations implements 
HiveOperationsBase {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HiveViewOperations.class);
+
+  private final String fullName;
+  private final String database;
+  private final String viewName;
+  private final FileIO fileIO;
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final long maxHiveTablePropertySize;
+
+  HiveViewOperations(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      FileIO fileIO,
+      String catalogName,
+      TableIdentifier viewIdentifier) {
+    String dbName = viewIdentifier.namespace().level(0);
+    this.metaClients = metaClients;
+    this.fileIO = fileIO;
+    this.fullName = BaseMetastoreCatalog.fullTableName(catalogName, 
viewIdentifier);
+    this.database = dbName;
+    this.viewName = viewIdentifier.name();
+    this.maxHiveTablePropertySize =
+        conf.getLong(HIVE_TABLE_PROPERTY_MAX_SIZE, 
HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT);
+  }
+
+  @Override
+  public void doRefresh() {
+    String metadataLocation = null;
+    Table table = null;
+
+    try {
+      table = metaClients.run(client -> client.getTable(database, viewName));
+      HiveOperationsBase.validateTableOrViewIsIceberg(table, fullName);
+      metadataLocation =
+          
table.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP);
+
+    } catch (NoSuchObjectException e) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchViewException("View does not exist: %s.%s", database, 
viewName);
+      }
+    } catch (TException e) {
+      String errMsg =
+          String.format("Failed to get view info from metastore %s.%s", 
database, viewName);
+      throw new RuntimeException(errMsg, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    }
+
+    if (table != null && 
!table.getTableType().equalsIgnoreCase(TableType.VIRTUAL_VIEW.name())) {
+      disableRefresh();
+    } else {
+      refreshFromMetadataLocation(metadataLocation);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")

Review Comment:
   Would it worth to refactor this method to decrease the 
`CyclomaticComplexity`?



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:
##########
@@ -264,6 +257,159 @@ public void renameTable(TableIdentifier from, 
TableIdentifier originalTo) {
     }
   }
 
+  @Override
+  public boolean dropView(TableIdentifier identifier) {
+    if (!isValidIdentifier(identifier)) {
+      return false;
+    }
+
+    try {
+      String database = identifier.namespace().level(0);
+      String viewName = identifier.name();
+      Table table = clients.run(client -> client.getTable(database, viewName));
+      HiveViewOperations.validateTableIsIcebergView(table, fullTableName(name, 
identifier));
+      clients.run(
+          client -> {
+            client.dropTable(database, viewName);
+            return null;
+          });
+      LOG.info("Dropped View: {}", identifier);
+      return true;
+
+    } catch (NoSuchViewException | NoSuchObjectException e) {
+      LOG.info("Skipping drop, View does not exist: {}", identifier, e);
+      return false;
+    } catch (TException e) {
+      throw new RuntimeException("Failed to drop " + identifier, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropView", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listViews(Namespace namespace) {
+    Preconditions.checkArgument(
+        isValidateNamespace(namespace), "Missing database in namespace: %s", 
namespace);
+
+    try {
+      return listTablesByType(
+          namespace, TableType.VIRTUAL_VIEW, 
BaseMetastoreTableOperations.ICEBERG_VIEW_TYPE_VALUE);
+    } catch (UnknownDBException e) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", 
namespace);
+
+    } catch (TException e) {
+      throw new RuntimeException("Failed to list all views under namespace " + 
namespace, e);
+
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to listViews", e);
+    }
+  }
+
+  private List<TableIdentifier> listAllTables(Namespace namespace)
+      throws TException, InterruptedException {
+    String database = namespace.level(0);
+    List<String> tableNames = clients.run(client -> 
client.getAllTables(database));
+    return tableNames.stream()
+        .map(t -> TableIdentifier.of(namespace, t))
+        .collect(Collectors.toList());
+  }
+
+  private List<TableIdentifier> listTablesByType(
+      Namespace namespace, TableType tableType, String tableTypeProp)
+      throws TException, InterruptedException {
+    String database = namespace.level(0);
+    List<String> tableNames = clients.run(client -> client.getTables(database, 
"*", tableType));
+    // Each TableData is a big object.Retrieving it with a big number can put 
pressure on memory.
+    // To be safe, it is being retired in batches.

Review Comment:
   Maybe:
   ```
   // Retrieving the Table objects from HMS in batches to avoid OOM
   ```



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:
##########
@@ -264,6 +265,163 @@ public void renameTable(TableIdentifier from, 
TableIdentifier originalTo) {
     }
   }
 
+  @Override
+  public boolean viewExists(TableIdentifier identifier) {
+    return HiveCatalogUtil.isTableWithTypeExists(clients, identifier, 
TableType.VIRTUAL_VIEW);
+  }
+
+  @Override
+  public boolean dropView(TableIdentifier identifier) {

Review Comment:
   The question still stands: Do we want to add `purge` to the dropView method 
too?



##########
core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java:
##########
@@ -307,24 +308,48 @@ protected enum CommitStatus {
    * @param newMetadataLocation the path of the new commit file
    * @param config metadata to use for configuration
    * @return Commit Status of Success, Failure or Unknown
+   * @deprecated since 1.5.0, use {@link 
BaseMetastoreTableOperations#checkCommitStatus(String,
+   *     String, Map, Function)} instead.
    */
+  @Deprecated

Review Comment:
   @nk1506: Please remove deprecation as per @rdblue comment.



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.iceberg.hive;
+
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Objects;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchIcebergViewException;
+import org.apache.iceberg.exceptions.NoSuchViewException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.util.MetastoreOperationsUtil;
+import org.apache.iceberg.view.BaseViewOperations;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Hive implementation of Iceberg ViewOperations. */
+final class HiveViewOperations extends BaseViewOperations implements 
HiveOperationsBase {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HiveViewOperations.class);
+
+  private final String fullName;
+  private final String database;
+  private final String viewName;
+  private final FileIO fileIO;
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final long maxHiveTablePropertySize;
+
+  HiveViewOperations(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      FileIO fileIO,
+      String catalogName,
+      TableIdentifier viewIdentifier) {
+    String dbName = viewIdentifier.namespace().level(0);
+    this.metaClients = metaClients;
+    this.fileIO = fileIO;
+    this.fullName = BaseMetastoreCatalog.fullTableName(catalogName, 
viewIdentifier);
+    this.database = dbName;
+    this.viewName = viewIdentifier.name();
+    this.maxHiveTablePropertySize =
+        conf.getLong(HIVE_TABLE_PROPERTY_MAX_SIZE, 
HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT);
+  }
+
+  @Override
+  public void doRefresh() {
+    String metadataLocation = null;
+    Table table = null;
+
+    try {
+      table = metaClients.run(client -> client.getTable(database, viewName));
+      HiveOperationsBase.validateTableOrViewIsIceberg(table, fullName);
+      metadataLocation =
+          
table.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP);
+
+    } catch (NoSuchObjectException e) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchViewException("View does not exist: %s.%s", database, 
viewName);
+      }
+    } catch (TException e) {
+      String errMsg =
+          String.format("Failed to get view info from metastore %s.%s", 
database, viewName);
+      throw new RuntimeException(errMsg, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    }
+
+    if (table != null && 
!table.getTableType().equalsIgnoreCase(TableType.VIRTUAL_VIEW.name())) {
+      disableRefresh();
+    } else {
+      refreshFromMetadataLocation(metadataLocation);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public void doCommit(ViewMetadata base, ViewMetadata metadata) {

Review Comment:
   What are the main differences between this method and the 
`HiveTableOperations.doCommit`? How much of the code is the same?



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java:
##########
@@ -181,4 +186,30 @@ default Table newHmsTable(String hmsTableOwner) {
 
     return newTable;
   }
+
+  default void setHmsParameters(
+      Table tbl,
+      String newMetadataLocation,
+      Schema schema,
+      String uuid,
+      Supplier<String> previousLocationSupplier) {
+    Map<String, String> parameters =
+        Optional.ofNullable(tbl.getParameters()).orElseGet(Maps::newHashMap);
+
+    if (uuid != null) {
+      parameters.put(TableProperties.UUID, uuid);
+    }
+    parameters.put(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, 
newMetadataLocation);
+    parameters.put(
+        BaseMetastoreTableOperations.TABLE_TYPE_PROP,
+        
BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ENGLISH));
+
+    if (previousLocationSupplier.get() != null && 
!previousLocationSupplier.get().isEmpty()) {
+      parameters.put(
+          BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP,
+          previousLocationSupplier.get());
+    }

Review Comment:
   Still needs fixing



##########
hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.iceberg.hive;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.view.ViewCatalogTests;
+import org.apache.thrift.TException;
+import org.assertj.core.api.Assumptions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+public class TestHiveViewCatalog extends ViewCatalogTests<HiveCatalog> {
+
+  private HiveCatalog catalog;
+
+  @RegisterExtension
+  private static final HiveMetastoreExtension HIVE_METASTORE_EXTENSION =
+      HiveMetastoreExtension.builder().build();
+
+  @BeforeEach
+  public void before() throws TException {
+    catalog =
+        (HiveCatalog)
+            CatalogUtil.loadCatalog(
+                HiveCatalog.class.getName(),
+                CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE,
+                ImmutableMap.of(
+                    CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS,
+                    String.valueOf(TimeUnit.SECONDS.toMillis(10))),
+                HIVE_METASTORE_EXTENSION.hiveConf());
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    HIVE_METASTORE_EXTENSION.metastore().reset();
+  }
+
+  @Override
+  protected HiveCatalog catalog() {
+    return catalog;
+  }
+
+  @Override
+  protected Catalog tableCatalog() {
+    return catalog;
+  }
+
+  @Override
+  protected boolean requiresNamespaceCreate() {
+    return true;
+  }
+
+  // TODO: This test should be removed after fix of 
https://github.com/apache/iceberg/issues/9289.
+  @Test
+  public void renameTableTargetAlreadyExistsAsView() {
+    Assumptions.assumeThat(tableCatalog())
+        .as("Only valid for catalogs that support tables")
+        .isNotNull();
+
+    TableIdentifier viewIdentifier = TableIdentifier.of("ns", "view");
+    TableIdentifier tableIdentifier = TableIdentifier.of("ns", "table");
+
+    if (requiresNamespaceCreate()) {
+      catalog().createNamespace(tableIdentifier.namespace());
+    }
+
+    assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should 
not exist").isFalse();
+
+    tableCatalog().buildTable(tableIdentifier, SCHEMA).create();
+
+    assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should 
exist").isTrue();
+
+    assertThat(catalog().viewExists(viewIdentifier)).as("View should not 
exist").isFalse();
+
+    catalog()
+        .buildView(viewIdentifier)
+        .withSchema(SCHEMA)
+        .withDefaultNamespace(viewIdentifier.namespace())
+        .withQuery("spark", "select * from ns.tbl")
+        .create();
+
+    assertThat(catalog().viewExists(viewIdentifier)).as("View should 
exist").isTrue();
+
+    // With fix of issues#9289,it should match with ViewCatalogTests and expect

Review Comment:
   I would like to see the `RuntimeException`s removed from the exceptions 
thrown from the `HiveCatalog` rename operations. I do not think that the issue 
should be solved by generally removing the `RuntimeException` wrapping, but we 
should unwrap it in the `HiveCatalog`.
   If we do this before merging the PR, then we will have a better code



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -162,8 +163,11 @@ protected void doRefresh() {
       Thread.currentThread().interrupt();
       throw new RuntimeException("Interrupted during refresh", e);
     }
-
-    refreshFromMetadataLocation(metadataLocation, metadataRefreshMaxRetries);
+    if (table != null && 
table.getTableType().equalsIgnoreCase(TableType.VIRTUAL_VIEW.name())) {
+      disableRefresh();

Review Comment:
   What is the exception that is thrown here using the `InMemoryCatalog`?
   Is it the correct behaviour for the `InMemoryCatalog`?



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.iceberg.hive;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.NoSuchViewException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hive.HiveCatalogUtil.CommitStatus;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.view.BaseViewOperations;
+import org.apache.iceberg.view.ViewMetadata;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Hive implementation of Iceberg ViewOperations. */
+final class HiveViewOperations extends BaseViewOperations implements 
HiveOperationsBase {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HiveViewOperations.class);
+
+  private final String fullName;
+  private final String database;
+  private final String viewName;
+  private final FileIO fileIO;
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final long maxHiveTablePropertySize;
+
+  HiveViewOperations(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      FileIO fileIO,
+      String catalogName,
+      TableIdentifier viewIdentifier) {
+    String dbName = viewIdentifier.namespace().level(0);
+    this.metaClients = metaClients;
+    this.fileIO = fileIO;
+    this.fullName = catalogName + "." + dbName + "." + viewIdentifier.name();
+    this.database = dbName;
+    this.viewName = viewIdentifier.name();
+    this.maxHiveTablePropertySize =
+        conf.getLong(HIVE_TABLE_PROPERTY_MAX_SIZE, 
HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT);
+  }
+
+  @Override
+  public void doRefresh() {
+    String metadataLocation = null;
+    Table table = null;
+
+    try {
+      table = metaClients.run(client -> client.getTable(database, viewName));
+      HiveOperationsBase.validateTableIsIceberg(table, fullName);
+      metadataLocation =
+          
table.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP);
+
+    } catch (NoSuchObjectException e) {
+      if (currentMetadataLocation() != null) {
+        throw new NoSuchViewException("View does not exist: %s.%s", database, 
viewName);
+      }
+    } catch (TException e) {
+      String errMsg =
+          String.format("Failed to get view info from metastore %s.%s", 
database, viewName);
+      throw new RuntimeException(errMsg, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    }
+
+    if (table != null && 
!table.getTableType().equalsIgnoreCase(TableType.VIRTUAL_VIEW.name())) {
+      disableRefresh();

Review Comment:
   I think here we should throw an exception if the type is not view...
   Maybe we should check if the `InMemoryCatalog` handles this case 
incorrectly, and fix this issue there in a different PR



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to