You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/09/24 17:41:48 UTC

[GitHub] [iceberg] szehon-ho commented on a change in pull request #3099: Core: Switch to RetryingHMSClient (allows configuration of retryDelays and retries)

szehon-ho commented on a change in pull request #3099:
URL: https://github.com/apache/iceberg/pull/3099#discussion_r715796353



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java
##########
@@ -110,57 +103,4 @@ public void testGetTablesFailsForNonReconnectableException() throws Exception {
     AssertHelpers.assertThrows("Should throw exception", MetaException.class,
             "Another meta exception", () -> clients.run(client -> client.getTables("default", "t")));
   }
-
-  @Test
-  public void testConnectionFailureRestoreForMetaException() throws Exception {

Review comment:
       Good idea, done.  Put the test back.
   
   Side note: the flexibility of making it configurable is nice, and is a potential soution the issue I mentioned, I am tempted to set it to true in HiveClientOperations::acquireLock in this line to address the issue I mentioned above.
   
   ```
   LockResponse lockResponse = metaClients.run(client -> client.lock(lockRequest));
   ```
   
   I can see from Hive point of view why they don't want to auto-retry, but most exception scenarios are not going to be in the corner case of having succeeded in the backend and leaving the lock.  In all honesty, if the Iceberg job fails because lock() throws TTransportException, end user will just try again but it is a wasted effort.  Not sure if any thoughts on it? 
    cc @pvary 

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java
##########
@@ -21,22 +21,24 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.iceberg.ClientPoolImpl;
-import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.common.DynMethods;
 import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
 
-public class HiveClientPool extends ClientPoolImpl<HiveMetaStoreClient, TException> {
+public class HiveClientPool extends ClientPoolImpl<IMetaStoreClient, TException> {
 
-  // use appropriate ctor depending on whether we're working with Hive2 or Hive3 dependencies
-  // we need to do this because there is a breaking API change between Hive2 and Hive3
-  private static final DynConstructors.Ctor<HiveMetaStoreClient> CLIENT_CTOR = DynConstructors.builder()
-          .impl(HiveMetaStoreClient.class, HiveConf.class)
-          .impl(HiveMetaStoreClient.class, Configuration.class)
-          .build();
+  // use appropriate ctor depending on whether we're working with Hive1, Hive2, or Hive3 dependencies
+  // we need to do this because there is a breaking API change between Hive1, Hive2, and Hive3
+  private static final DynMethods.StaticMethod CLIENT_CTOR = DynMethods.builder("getProxy")

Review comment:
       Done




-- 
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-unsubscribe@iceberg.apache.org

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



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