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 2023/01/12 10:18:28 UTC

[GitHub] [iceberg] pvary opened a new pull request, #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

pvary opened a new pull request, #6570:
URL: https://github.com/apache/iceberg/pull/6570

   HIVE-26882 gives us the possibility to atomically change the `metadata_location` using a single `alter_table` HMS call. The change is a new feature, so exception is needed from the Hive community to include it to new releases. OTOH I deliberately kept simple, so if someone uses their own Hive release they could backport the change easily.
   
   If we start using this possibility we can avoid using HMS locks to ensure the atomicity of the `HiveTableOperations.commit`. This has the following benefits:
   - Instead of 4 HMS calls (lock, getTable, alter_table, unlock) we can use only 2 calls (getTable, alter_table) which could help in commit performance.
   - I have seen several complains when the locks were not removed correctly (#3336, #2301) and suggested solutions (#6370). Also did my share to fix the situation as much as possible too (#6451)
   - Removing the locks would remove much of this complexity
   
   The solution has 2 parts:
   - I rebased and refreshed @szlta's work on #5877 to refactor out the Lock related code to its own class
   - Added a new feature to disable the locking mechanism
   
   Some of my concerns and thoughts:
   - I am not sure when HIVE-26882 will be available in OS. Here I would ask help from the Hive folks: @InvisibleProgrammer, @TuroczyX, @deniskuzZ.
       - The simplicity of the Hive PR helps alleviating my fears here
   - We already have `LockManager` interface defined in the `iceberg-api` module. After some back-and-forth I decided against using it, because of the following reasons:
       - I do not think anyone would like to use HiveLockManager without HiveCatalog
       - The interface is not that useful for us:
           - We would need to keep track of the HMS lockId internally
           - We would need to update the LockManager if the `setConf` method is called on the HiveCatalog
           - We would need to add something like `ensureActive` to the interface which is needed for HiveTableOperations
       - The `BaseLockManager` does not provide too much of a functionality
       - The current configuration keys are different from the ones used by the `LockManager` implementations
   
   WDYT?
   Open for comments and suggestions.
   
   Thanks,
   Peter


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1160060520


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   That sounds ok if its a double-check, but the way I read this  code, if table config is set to NO_LOCK_ENABLED, it doesnt matter what the writer sets?   So looks like table config is all that matters. 
   
   Should we improve this check to make sure both writer/table config is NO_LOCK?



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   It would be great if we could at least protect the user from this potential corruption error (running without the patch HIVE-26882)
   
   General question: is there any way we can check Hive server version?  Would be nice in general.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1071920474


##########
docs/configuration.md:
##########
@@ -175,8 +175,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.lock.hive.enabled                 | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  | 
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.lock.hive.enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)

Review Comment:
   On HMS side HIVE-26882 is needed on both HMS nodes.
   This config value is on the Iceberg side (client side from the HMS point of view).
   
   But the question is good, because it highlights an important thing:
   - All of the Iceberg writers should have the same configuration
   
   This is the exact reason why I have added the table level configuration, so we can change the behaviour of every clients "atomically" by changing the table property.
   So the upgrade process would look like this:
   - Upgrade all of the clients to have the new code
   - Change the table property to turn off the locking
   



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1081787456


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
+   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
+   * otherwise an appropriate exception is thrown.
+   *
+   * @return The created lock
+   * @throws LockException When we are not able to fill the hostname for lock creation, or there is
+   *     an error during lock creation
+   */
+  @SuppressWarnings("ReverseDnsLookup")
+  private LockInfo tryLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    final LockComponent lockComponent =
+        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    final LockRequest lockRequest =
+        new LockRequest(
+            Lists.newArrayList(lockComponent), System.getProperty("user.name"), hostName);
+
+    // Only works in Hive 2 or later.
+    if (HiveVersion.min(HiveVersion.HIVE_2)) {
+      lockRequest.setAgentInfo(agentInfo);
+    }
+
+    AtomicBoolean interrupted = new AtomicBoolean(false);
+    Tasks.foreach(lockRequest)
+        .retry(Integer.MAX_VALUE - 100)
+        .exponentialBackoff(
+            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
+        .shouldRetryTest(e -> !interrupted.get() && HiveVersion.min(HiveVersion.HIVE_2))

Review Comment:
   No need to keep e instanceof TException here, as in the old code?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1095799391


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
+   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
+   * otherwise an appropriate exception is thrown.
+   *
+   * @return The created lock
+   * @throws LockException When we are not able to fill the hostname for lock creation, or there is
+   *     an error during lock creation
+   */
+  @SuppressWarnings("ReverseDnsLookup")
+  private LockInfo tryLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    final LockComponent lockComponent =
+        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    final LockRequest lockRequest =
+        new LockRequest(
+            Lists.newArrayList(lockComponent), System.getProperty("user.name"), hostName);

Review Comment:
   Done in #6648 



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


[GitHub] [iceberg] pvary commented on pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-1415973878

   @szehon-ho, @RussellSpitzer: Updated after the merge of the lock refactor.
   This is a much smaller/straightforward change now.
   Could you please review?


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


Re: [PR] Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882 [iceberg]

Posted by "chenwyi2 (via GitHub)" <gi...@apache.org>.
chenwyi2 commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-2024332961

   https://issues.apache.org/jira/browse/HIVE-26882 has something wrong, should we has some documentation to info user?


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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1071920742


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -809,168 +603,38 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
   }
 
   /**
-   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
-   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
-   * otherwise an appropriate exception is thrown.
+   * Returns if the hive locking should be enabled on the table, or not.
    *
-   * @param agentInfo The agentInfo which should be used during lock creation
-   * @return The created lock
-   * @throws UnknownHostException When we are not able to fill the hostname for lock creation
-   * @throws TException When there is an error during lock creation
-   */
-  @SuppressWarnings("ReverseDnsLookup")
-  private LockInfo tryLock(String agentInfo) throws UnknownHostException, TException {
-    LockInfo lockInfo = new LockInfo();
-
-    final LockComponent lockComponent =
-        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, database);
-    lockComponent.setTablename(tableName);
-    final LockRequest lockRequest =
-        new LockRequest(
-            Lists.newArrayList(lockComponent),
-            System.getProperty("user.name"),
-            InetAddress.getLocalHost().getHostName());
-
-    // Only works in Hive 2 or later.
-    if (HiveVersion.min(HiveVersion.HIVE_2)) {
-      lockRequest.setAgentInfo(agentInfo);
-    }
-
-    Tasks.foreach(lockRequest)
-        .retry(Integer.MAX_VALUE - 100)
-        .exponentialBackoff(
-            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
-        .shouldRetryTest(e -> e instanceof TException && HiveVersion.min(HiveVersion.HIVE_2))
-        .throwFailureWhenFinished()
-        .run(
-            request -> {
-              try {
-                LockResponse lockResponse = metaClients.run(client -> client.lock(request));
-                lockInfo.lockId = lockResponse.getLockid();
-                lockInfo.lockState = lockResponse.getState();
-              } catch (TException te) {
-                LOG.warn("Failed to acquire lock {}", request, te);
-                try {
-                  // If we can not check for lock, or we do not find it, then rethrow the exception
-                  // Otherwise we are happy as the findLock sets the lockId and the state correctly
-                  if (!HiveVersion.min(HiveVersion.HIVE_2)) {
-                    LockInfo lockFound = findLock(agentInfo);
-                    if (lockFound != null) {
-                      lockInfo.lockId = lockFound.lockId;
-                      lockInfo.lockState = lockFound.lockState;
-                      LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
-                      return;
-                    }
-                  }
-
-                  throw te;
-                } catch (InterruptedException e) {
-                  Thread.currentThread().interrupt();
-                  LOG.warn(
-                      "Interrupted while checking for lock on table {}.{}", database, tableName, e);
-                  throw new RuntimeException("Interrupted while checking for lock", e);
-                }
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                LOG.warn("Interrupted while acquiring lock on table {}.{}", database, tableName, e);
-                throw new RuntimeException("Interrupted while acquiring lock", e);
-              }
-            },
-            TException.class);
-
-    // This should be initialized always, or exception should be thrown.
-    LOG.debug("Lock {} created for table {}.{}", lockInfo, database, tableName);
-    return lockInfo;
-  }
-
-  /**
-   * Search for the locks using HMSClient.showLocks identified by the agentInfo. If the lock is
-   * there, then a {@link LockInfo} object is returned. If the lock is not found <code>null</code>
-   * is returned.
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#LOCK_HIVE_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#LOCK_HIVE_ENABLED_DEFAULT}
+   * </ol>
    *
-   * @param agentInfo The key for searching the locks
-   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
    */
-  private LockInfo findLock(String agentInfo) throws TException, InterruptedException {
-    Preconditions.checkArgument(
-        HiveVersion.min(HiveVersion.HIVE_2),
-        "Minimally Hive 2 HMS client is needed to find the Lock using the showLocks API call");
-    ShowLocksRequest showLocksRequest = new ShowLocksRequest();
-    showLocksRequest.setDbname(database);
-    showLocksRequest.setTablename(tableName);
-    ShowLocksResponse response = metaClients.run(client -> client.showLocks(showLocksRequest));
-    for (ShowLocksResponseElement lock : response.getLocks()) {
-      if (lock.getAgentInfo().equals(agentInfo)) {
-        // We found our lock
-        return new LockInfo(lock.getLockid(), lock.getState());
-      }
-    }
-
-    // Not found anything
-    return null;
-  }
-
-  private static class HiveLockHeartbeat implements Runnable {
-    private final ClientPool<IMetaStoreClient, TException> hmsClients;
-    private final long lockId;
-    private final long intervalMs;
-    private ScheduledFuture<?> future;
-    private volatile Exception encounteredException = null;
-
-    HiveLockHeartbeat(
-        ClientPool<IMetaStoreClient, TException> hmsClients, long lockId, long intervalMs) {
-      this.hmsClients = hmsClients;
-      this.lockId = lockId;
-      this.intervalMs = intervalMs;
-      this.future = null;
-    }
-
-    @Override
-    public void run() {
-      try {
-        hmsClients.run(
-            client -> {
-              client.heartbeat(0, lockId);
-              return null;
-            });
-      } catch (TException | InterruptedException e) {
-        this.encounteredException = e;
-        throw new CommitFailedException(e, "Failed to heartbeat for lock: %d", lockId);
-      }
-    }
-
-    public void schedule(ScheduledExecutorService scheduler) {
-      future =
-          scheduler.scheduleAtFixedRate(this, intervalMs / 2, intervalMs, TimeUnit.MILLISECONDS);
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {
+    if (metadata.properties().get(TableProperties.LOCK_HIVE_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.LOCK_HIVE_ENABLED, false);
     }
 
-    public void cancel() {
-      if (future != null) {
-        future.cancel(false);
-      }
-    }
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.LOCK_HIVE_ENABLED_DEFAULT);
   }
 
-  private static class LockInfo {
-    private long lockId;
-    private LockState lockState;
-
-    private LockInfo() {
-      this.lockId = -1;
-      this.lockState = null;
-    }
-
-    private LockInfo(long lockId, LockState lockState) {
-      this.lockId = lockId;
-      this.lockState = lockState;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("lockId", lockId)
-          .add("lockState", lockState)
-          .toString();
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName);

Review Comment:
   Yes, you are right



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1073150734


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();

Review Comment:
   FWIW this is the same behavior that was before this change.
   
   If the jvm is serving multiple queries then this could happen. IIRC this was suggested by @RussellSpitzer and come from the Spark side



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1071927823


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -809,168 +603,38 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
   }
 
   /**
-   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
-   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
-   * otherwise an appropriate exception is thrown.
+   * Returns if the hive locking should be enabled on the table, or not.
    *
-   * @param agentInfo The agentInfo which should be used during lock creation
-   * @return The created lock
-   * @throws UnknownHostException When we are not able to fill the hostname for lock creation
-   * @throws TException When there is an error during lock creation
-   */
-  @SuppressWarnings("ReverseDnsLookup")
-  private LockInfo tryLock(String agentInfo) throws UnknownHostException, TException {
-    LockInfo lockInfo = new LockInfo();
-
-    final LockComponent lockComponent =
-        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, database);
-    lockComponent.setTablename(tableName);
-    final LockRequest lockRequest =
-        new LockRequest(
-            Lists.newArrayList(lockComponent),
-            System.getProperty("user.name"),
-            InetAddress.getLocalHost().getHostName());
-
-    // Only works in Hive 2 or later.
-    if (HiveVersion.min(HiveVersion.HIVE_2)) {
-      lockRequest.setAgentInfo(agentInfo);
-    }
-
-    Tasks.foreach(lockRequest)
-        .retry(Integer.MAX_VALUE - 100)
-        .exponentialBackoff(
-            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
-        .shouldRetryTest(e -> e instanceof TException && HiveVersion.min(HiveVersion.HIVE_2))
-        .throwFailureWhenFinished()
-        .run(
-            request -> {
-              try {
-                LockResponse lockResponse = metaClients.run(client -> client.lock(request));
-                lockInfo.lockId = lockResponse.getLockid();
-                lockInfo.lockState = lockResponse.getState();
-              } catch (TException te) {
-                LOG.warn("Failed to acquire lock {}", request, te);
-                try {
-                  // If we can not check for lock, or we do not find it, then rethrow the exception
-                  // Otherwise we are happy as the findLock sets the lockId and the state correctly
-                  if (!HiveVersion.min(HiveVersion.HIVE_2)) {
-                    LockInfo lockFound = findLock(agentInfo);
-                    if (lockFound != null) {
-                      lockInfo.lockId = lockFound.lockId;
-                      lockInfo.lockState = lockFound.lockState;
-                      LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
-                      return;
-                    }
-                  }
-
-                  throw te;
-                } catch (InterruptedException e) {
-                  Thread.currentThread().interrupt();
-                  LOG.warn(
-                      "Interrupted while checking for lock on table {}.{}", database, tableName, e);
-                  throw new RuntimeException("Interrupted while checking for lock", e);
-                }
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                LOG.warn("Interrupted while acquiring lock on table {}.{}", database, tableName, e);
-                throw new RuntimeException("Interrupted while acquiring lock", e);
-              }
-            },
-            TException.class);
-
-    // This should be initialized always, or exception should be thrown.
-    LOG.debug("Lock {} created for table {}.{}", lockInfo, database, tableName);
-    return lockInfo;
-  }
-
-  /**
-   * Search for the locks using HMSClient.showLocks identified by the agentInfo. If the lock is
-   * there, then a {@link LockInfo} object is returned. If the lock is not found <code>null</code>
-   * is returned.
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#LOCK_HIVE_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#LOCK_HIVE_ENABLED_DEFAULT}
+   * </ol>
    *
-   * @param agentInfo The key for searching the locks
-   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
    */
-  private LockInfo findLock(String agentInfo) throws TException, InterruptedException {
-    Preconditions.checkArgument(
-        HiveVersion.min(HiveVersion.HIVE_2),
-        "Minimally Hive 2 HMS client is needed to find the Lock using the showLocks API call");
-    ShowLocksRequest showLocksRequest = new ShowLocksRequest();
-    showLocksRequest.setDbname(database);
-    showLocksRequest.setTablename(tableName);
-    ShowLocksResponse response = metaClients.run(client -> client.showLocks(showLocksRequest));
-    for (ShowLocksResponseElement lock : response.getLocks()) {
-      if (lock.getAgentInfo().equals(agentInfo)) {
-        // We found our lock
-        return new LockInfo(lock.getLockid(), lock.getState());
-      }
-    }
-
-    // Not found anything
-    return null;
-  }
-
-  private static class HiveLockHeartbeat implements Runnable {
-    private final ClientPool<IMetaStoreClient, TException> hmsClients;
-    private final long lockId;
-    private final long intervalMs;
-    private ScheduledFuture<?> future;
-    private volatile Exception encounteredException = null;
-
-    HiveLockHeartbeat(
-        ClientPool<IMetaStoreClient, TException> hmsClients, long lockId, long intervalMs) {
-      this.hmsClients = hmsClients;
-      this.lockId = lockId;
-      this.intervalMs = intervalMs;
-      this.future = null;
-    }
-
-    @Override
-    public void run() {
-      try {
-        hmsClients.run(
-            client -> {
-              client.heartbeat(0, lockId);
-              return null;
-            });
-      } catch (TException | InterruptedException e) {
-        this.encounteredException = e;
-        throw new CommitFailedException(e, "Failed to heartbeat for lock: %d", lockId);
-      }
-    }
-
-    public void schedule(ScheduledExecutorService scheduler) {
-      future =
-          scheduler.scheduleAtFixedRate(this, intervalMs / 2, intervalMs, TimeUnit.MILLISECONDS);
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {
+    if (metadata.properties().get(TableProperties.LOCK_HIVE_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.LOCK_HIVE_ENABLED, false);
     }
 
-    public void cancel() {
-      if (future != null) {
-        future.cancel(false);
-      }
-    }
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.LOCK_HIVE_ENABLED_DEFAULT);
   }
 
-  private static class LockInfo {
-    private long lockId;
-    private LockState lockState;
-
-    private LockInfo() {
-      this.lockId = -1;
-      this.lockState = null;
-    }
-
-    private LockInfo(long lockId, LockState lockState) {
-      this.lockId = lockId;
-      this.lockState = lockState;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("lockId", lockId)
-          .add("lockState", lockState)
-          .toString();
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {

Review Comment:
   This is a thing I was debating a lot.
   
   The situation is that we already have a `LockManager` interface which is used by other catalogs, but which is not appropriate for us. See, the PR description for more details.
   
   IMHO if you are using HiveCatalog, you should stick to Hive for making sure that the Iceberg commit is atomic. So I do not see multiple implementation in the future for our `OtherLockManager` interface. Also creating another interface for `LockManager` would be confusing at best.
   
   These were the reasons behind my decision, but I am not strongly convinced in any case.



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

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


[GitHub] [iceberg] pvary merged pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary merged PR #6570:
URL: https://github.com/apache/iceberg/pull/6570


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


[GitHub] [iceberg] szehon-ho commented on pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-1518360711

   Hey thanks, I had two additional comments for consideration, otherwise it looks good to me.


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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1174885729


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   We always add `ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE)` which is an optimisation for skipping stats generation / file listing when it is not needed on alter table commands. This always depended on DynMethod skipping last parameters (maybe by accident)
   
   See: https://github.com/apache/iceberg/blob/fede493d59f17ff2bfc0744b296d90bd36130386/hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java#L56-L59
   
   We can modify the proposed command to:
   ```
   if (env.size() > 1) {
     Preconditions.checkArgument("Environment context is non-empty but alter_table method does not support it", 
     ALTER_TABLE.args().size() == 5);
   }
   ```
   
   Or add a new check to the the new `alterTable` method, but I find it better / more readable to add the check to the NoLock constructor.
   
   WDYT?



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


[GitHub] [iceberg] haizhou-zhao commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
haizhou-zhao commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1072885938


##########
docs/configuration.md:
##########
@@ -175,8 +175,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.lock.hive.enabled                 | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  | 
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.lock.hive.enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)

Review Comment:
   Do you know if there's plan to backport HIVE-26882 into previous Hive major releases, or is this only present on the newest major version (Hive 4)?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1073151680


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/NoLock.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+public class NoLock implements HiveLock {

Review Comment:
   Based on the configuration of the `Catalog` and the `Table`  the lock could be either `NoLock` or `MetastoreLock`



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1095798506


##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String LOCK_HIVE_ENABLED = "lock.hive.enabled";

Review Comment:
   Renamed



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159969613


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -263,6 +266,15 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
         throw e;
 
       } catch (Throwable e) {
+        if (e.getMessage()

Review Comment:
   IIRC Thrift does not work well with subclassing



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1160038384


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -263,6 +266,15 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
         throw e;
 
       } catch (Throwable e) {
+        if (e.getMessage()

Review Comment:
   Hm. how about NoSuchObjectException ?  That seems to propagate to Iceberg code somehow?  



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


Re: [PR] Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882 [iceberg]

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-2025231799

   @chenwyi2: Maybe we should wait a bit and see if we have a fix and document that. 


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1161989553


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Oh maybe I was not clear, I meant the alter_method that we load dynamically:  https://github.com/apache/iceberg/blob/master/hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java#L29  .  Original question was just asking: if our method object is the one loaded without env, and we pass in env here, what will happen? (is the error message good enough for user to decipher, and it wont silently pass?)



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


[GitHub] [iceberg] pvary commented on pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-1521269536

   Thanks for the review @deniskuzZ, @rdblue, @haizhou-zhao!
   And special thanks for @szehon-ho!


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1160043614


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   It would be great if we could at least protect the user from this potential error (running without the patch HIVE-26882)
   
   General question: is there any way we can check Hive server version?  Would be nice in general.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159952096


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Sadly no 😢 . Historical reasons again



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159973960


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.

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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159974754


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental

Review Comment:
   Done.
   Thanks!



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


Re: [PR] Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882 [iceberg]

Posted by "kmensah-stripe (via GitHub)" <gi...@apache.org>.
kmensah-stripe commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-1944371875

   Looking at https://home.corp.stripe.com/compass/projects/removing-locking-for-iceberg-commits-in-hms I don't have context on why we're pulling in patches instead of upgrading. Not a blocker but would be helpful context


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


[GitHub] [iceberg] pvary commented on pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-1422053720

   @szehon-ho: Gentle reminder, could you please take a look when you have some time?
   Thanks, Peter 


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1081768050


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -53,9 +55,23 @@ private MetastoreUtil() {}
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If possible, an environmental context will

Review Comment:
   Nit: Maybe the stats update part should be put as an inline comment instead of the method?  Also is it 'if possible'?   It seems its always used.



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired

Review Comment:
   Style: Realize some people like to avoid else block, but to avoid the repetitive comments, is it easier to just put all these extra checks as part of the else ?  Then it would just be obvious from the code.



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock

Review Comment:
   Nit: "Creates a lock, retrying if possible on failure"?  More concise.
   
   Also I think we can remove "If the lock creation is successful then a {@link LockInfo} is returned,
      * otherwise an appropriate exception is thrown.", and move corresponding information to return , and throws elements, which are already in the comment.



##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String LOCK_HIVE_ENABLED = "lock.hive.enabled";

Review Comment:
   I also vote hive.lock.enabled as it sounds better, and it's more consistent with other Hadoop configuration for Iceberg-hive (iceberg.hive.xxx)



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
+   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
+   * otherwise an appropriate exception is thrown.
+   *
+   * @return The created lock
+   * @throws LockException When we are not able to fill the hostname for lock creation, or there is
+   *     an error during lock creation
+   */
+  @SuppressWarnings("ReverseDnsLookup")
+  private LockInfo tryLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    final LockComponent lockComponent =

Review Comment:
   Realize its moved code, but would be great to remove unnecessary 'final' here, (always get this comment from @aokolnychyi , as its apparently already automatically done by most JVMs)



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
+   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
+   * otherwise an appropriate exception is thrown.
+   *
+   * @return The created lock
+   * @throws LockException When we are not able to fill the hostname for lock creation, or there is
+   *     an error during lock creation
+   */
+  @SuppressWarnings("ReverseDnsLookup")
+  private LockInfo tryLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    final LockComponent lockComponent =
+        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    final LockRequest lockRequest =
+        new LockRequest(
+            Lists.newArrayList(lockComponent), System.getProperty("user.name"), hostName);
+
+    // Only works in Hive 2 or later.
+    if (HiveVersion.min(HiveVersion.HIVE_2)) {
+      lockRequest.setAgentInfo(agentInfo);
+    }
+
+    AtomicBoolean interrupted = new AtomicBoolean(false);
+    Tasks.foreach(lockRequest)
+        .retry(Integer.MAX_VALUE - 100)
+        .exponentialBackoff(
+            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
+        .shouldRetryTest(e -> !interrupted.get() && HiveVersion.min(HiveVersion.HIVE_2))

Review Comment:
   No need to keep e instanceof TException here?



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


[GitHub] [iceberg] pvary commented on pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-1483743889

   Thanks @rdblue for the review. Fixed the config keys


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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1083878354


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired

Review Comment:
   Done in #6648 - please continue to review this part there



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1095800121


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -53,9 +55,23 @@ private MetastoreUtil() {}
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If possible, an environmental context will

Review Comment:
   Changed the comment. I hope it is better now



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1160060520


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   That sounds ok if its a double-check, but the way the code here looks, if table config is set to NO_LOCK_ENABLED, it doesnt matter what the writer sets?   So looks like table config is all that matters.  Should we improve this check to make sure both writer/table config is NO_LOCK, as you descirbed?



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159171448


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Why do we want to allow override the table property using conf?  I thought all writes should use the hive lock enabled, or all writers should not.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1171238063


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   > Remove catalog config altogether
   
   Then it is not possible to handle 2/3 use-cases (We either have to set the flag for every table, or rely on the default provided by the Iceberg source code - I think it is more useful for the user to provide an override possibility on catalog level too). I am open to debate the usefulness of this cases, but if we want to support them, then we need 2 levels of configuration.
   
   > Have Client A throw exception in this case (as I was mentioning, this may be better)
   
   We can throw an exception if the Table level and the Catalog level config is contradictory. In this case every table used by a Catalog instance should have the same locking configuration if it is set at least for one of the tables used.
   If we decide on this then I agree with you that there is no need to have a Catalog level configuration, as it is just for throwing exceptions in some cases.
   
   
   I am starting to feel that the main difference is:
   - What do we consider more important:
       - Configuration flexibility: If the user wants to set all of the tables to use the new Locking mechanism, they should not need to go to every one of the tables and alter them one-by-one. A global catalog configuration should be enough in these cases. For me the question was how to handle when we would like to use a table differently than the general one and the table level config is proposed to do that.
       - Preventing wrong configurations: With the "only Table level config" solution we can be sure that if every writer uses which uses a code version which contains these changes will not have a writing conflict. BTW this can be archived even with the previous proposal if all of the tables are altered one-by-one to set the Table level config.
   



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1175798588


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Yea I was thinking at the very beginning of the method, before we put the ALTER_TABLE?  Yea it may be better for understanding of the code if we have both checks, but you are right it's not a big deal in practice, just in the rare case there's a patched version of Hive 2 out there that has alter_table without 5 arguments.



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


[GitHub] [iceberg] haizhou-zhao commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
haizhou-zhao commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1072875908


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();

Review Comment:
   For the sake of my understanding, do you mind elaborate a bit on what circumstances will we look at multiple thread from a single process trying to lock the same table simultaneously?



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1171963570


##########
docs/configuration.md:
##########
@@ -178,8 +178,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.engine.hive.lock-enabled          | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  |

Review Comment:
   Can be shortened: Use HMS locks to ensure atomicity of commits
   
   ?



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   I was looking through DynMethod, there is no check and will just pass whatever args it can fit.  I think it will be nice to add a 'argLength' method DynMethod and then assert that argLength == 5 here, so ensure the env is not silently dropped.



##########
docs/configuration.md:
##########
@@ -178,8 +178,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.engine.hive.lock-enabled          | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  |
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.engine.hive.lock-enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
+is available on the Hive Metastore server and every writer of a given table is using Iceberg 1.2 or later.

Review Comment:
   We should change this to warn.  We also miss warning about some catalogs setting true and others setting false.  
   
   How about this to add some details.  I use HiveCatalog here, even though its not defined in the doc, otherwise its quite lengthy to continue the language- "catalog using Hive Metastore connector"
   
   ```
   Warn: Setting `iceberg.engine.hive.lock-enabled` will cause HiveCatalog to commit to tables without using Hive locks.  This should only be set to `false` if all the following conditions are met: 
   * [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
   is available on the Hive Metastore server
   * All other HiveCatalogs committing to tables that this HiveCatalog commits to are also on Iceberg 1.3 or later
   * All other HiveCatalogs committing to tables that this HiveCatalog commits to have also disabled Hive locks on commit.
   
   Failing to ensure these conditions risks corrupting the table.
   
   Even with `iceberg.engine.hive.lock-enabled` set to `false`, a HiveCatalog can still use locks for individual tables by setting the table property 'engine.hive.lock-enabled'='true'.  This is useful in the case where other HiveCatalogs cannot be upgraded and set to commit without using Hive locks.
   



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1171970109


##########
docs/configuration.md:
##########
@@ -178,8 +178,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.engine.hive.lock-enabled          | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  |
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.engine.hive.lock-enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
+is available on the Hive Metastore server and every writer of a given table is using Iceberg 1.2 or later.

Review Comment:
   We should change this to warn.  We also miss warning about some catalogs setting true and others setting false.  
   
   How about this to add some details.  I use HiveCatalog here, even though its not defined in the doc, otherwise its quite lengthy to continue the language- "catalog using Hive Metastore connector"
   
   
   Warn: Setting `iceberg.engine.hive.lock-enabled` will cause HiveCatalog to commit to tables without using Hive locks.  This should only be set to `false` if all following conditions are met: 
   * [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
   is available on the Hive Metastore server
   * All other HiveCatalogs committing to tables that this HiveCatalog commits to are also on Iceberg 1.3 or later
   * All other HiveCatalogs committing to tables that this HiveCatalog commits to have also disabled Hive locks on commit.
   
   Failing to ensure these conditions risks corrupting the table.
   
   Even with `iceberg.engine.hive.lock-enabled` set to `false`, a HiveCatalog can still use locks for individual tables by setting the table property 'engine.hive.lock-enabled'='true'.  This is useful in the case where other HiveCatalogs cannot be upgraded and set to commit without using Hive locks.
   



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1170482092


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   No I am not thinking of Iceberg like a service, but in my thought, indeed, the TableProperties are a single configuration persisted on the table, and the CatalogProperties is set by individual clients that can override.
   
   My concern in the previous comments was, say we have one table, written by clients A and B.  Table property for lock is not set.
   
   - Client A sets catalog property LOCK_HIVE_ENABLED = "false"
   - Client B has catalog property LOCK_HIVE_ENABLED = "true"
   
   This will make clients corrupt each other commit.   So was suggesting, either 
   
   - Remove catalog config altogether
   - Have Client A throw exception in this case (as I was mentioning, this may be better)
   
   Does that make sense, or am I missing something?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1172265550


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   This method is used in other cases as well and doing some extra checks would be edgy.
   
   Added a check for the `NoLock` constructor to ensure that at least Hive 2 client is used when a `NoLock` object is created. This ensures that the correct client is used in this case.



##########
docs/configuration.md:
##########
@@ -178,8 +178,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.engine.hive.lock-enabled          | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  |

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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1172266345


##########
docs/configuration.md:
##########
@@ -178,8 +178,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.engine.hive.lock-enabled          | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  |
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.engine.hive.lock-enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
+is available on the Hive Metastore server and every writer of a given table is using Iceberg 1.2 or later.

Review Comment:
   Done.
   
   Added extra highlights for the warning `Failing to ensure these conditions risks corrupting the table.`



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159950604


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Since there is no official release with the new code yet, sadly this is not an option 😢 



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1160043614


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   General question: is there any way we can check Hive server version?  Would be nice in general.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1169474452


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   I think I finally understand our differences here.
   
   - You think about Iceberg as a service, and the users as clients. In this view we have service level configurations (TableProperties), and client level configurations (CatalogProperties)
   - I think about Iceberg as a library which provides different levels of configuration. The configuration provided by the developer (CatalogProperties), and global overrides which prevent configuration discrepancies between different users (TableProperties)
   
   Does this make sense?
   Am I right in understanding the different point of views?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1148316854


##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String HIVE_LOCK_ENABLED = "hive.lock.enabled";
+  public static final boolean HIVE_LOCK_ENABLED_DEFAULT = true;

Review Comment:
   Done 



##########
core/src/main/java/org/apache/iceberg/hadoop/ConfigProperties.java:
##########
@@ -23,5 +23,6 @@ public class ConfigProperties {
   private ConfigProperties() {}
 
   public static final String ENGINE_HIVE_ENABLED = "iceberg.engine.hive.enabled";
+  public static final String LOCK_HIVE_ENABLED = "iceberg.lock.hive.enabled";

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


[GitHub] [iceberg] deniskuzZ commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1071413371


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -809,168 +603,38 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
   }
 
   /**
-   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
-   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
-   * otherwise an appropriate exception is thrown.
+   * Returns if the hive locking should be enabled on the table, or not.
    *
-   * @param agentInfo The agentInfo which should be used during lock creation
-   * @return The created lock
-   * @throws UnknownHostException When we are not able to fill the hostname for lock creation
-   * @throws TException When there is an error during lock creation
-   */
-  @SuppressWarnings("ReverseDnsLookup")
-  private LockInfo tryLock(String agentInfo) throws UnknownHostException, TException {
-    LockInfo lockInfo = new LockInfo();
-
-    final LockComponent lockComponent =
-        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, database);
-    lockComponent.setTablename(tableName);
-    final LockRequest lockRequest =
-        new LockRequest(
-            Lists.newArrayList(lockComponent),
-            System.getProperty("user.name"),
-            InetAddress.getLocalHost().getHostName());
-
-    // Only works in Hive 2 or later.
-    if (HiveVersion.min(HiveVersion.HIVE_2)) {
-      lockRequest.setAgentInfo(agentInfo);
-    }
-
-    Tasks.foreach(lockRequest)
-        .retry(Integer.MAX_VALUE - 100)
-        .exponentialBackoff(
-            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
-        .shouldRetryTest(e -> e instanceof TException && HiveVersion.min(HiveVersion.HIVE_2))
-        .throwFailureWhenFinished()
-        .run(
-            request -> {
-              try {
-                LockResponse lockResponse = metaClients.run(client -> client.lock(request));
-                lockInfo.lockId = lockResponse.getLockid();
-                lockInfo.lockState = lockResponse.getState();
-              } catch (TException te) {
-                LOG.warn("Failed to acquire lock {}", request, te);
-                try {
-                  // If we can not check for lock, or we do not find it, then rethrow the exception
-                  // Otherwise we are happy as the findLock sets the lockId and the state correctly
-                  if (!HiveVersion.min(HiveVersion.HIVE_2)) {
-                    LockInfo lockFound = findLock(agentInfo);
-                    if (lockFound != null) {
-                      lockInfo.lockId = lockFound.lockId;
-                      lockInfo.lockState = lockFound.lockState;
-                      LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
-                      return;
-                    }
-                  }
-
-                  throw te;
-                } catch (InterruptedException e) {
-                  Thread.currentThread().interrupt();
-                  LOG.warn(
-                      "Interrupted while checking for lock on table {}.{}", database, tableName, e);
-                  throw new RuntimeException("Interrupted while checking for lock", e);
-                }
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                LOG.warn("Interrupted while acquiring lock on table {}.{}", database, tableName, e);
-                throw new RuntimeException("Interrupted while acquiring lock", e);
-              }
-            },
-            TException.class);
-
-    // This should be initialized always, or exception should be thrown.
-    LOG.debug("Lock {} created for table {}.{}", lockInfo, database, tableName);
-    return lockInfo;
-  }
-
-  /**
-   * Search for the locks using HMSClient.showLocks identified by the agentInfo. If the lock is
-   * there, then a {@link LockInfo} object is returned. If the lock is not found <code>null</code>
-   * is returned.
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#LOCK_HIVE_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#LOCK_HIVE_ENABLED_DEFAULT}
+   * </ol>
    *
-   * @param agentInfo The key for searching the locks
-   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
    */
-  private LockInfo findLock(String agentInfo) throws TException, InterruptedException {
-    Preconditions.checkArgument(
-        HiveVersion.min(HiveVersion.HIVE_2),
-        "Minimally Hive 2 HMS client is needed to find the Lock using the showLocks API call");
-    ShowLocksRequest showLocksRequest = new ShowLocksRequest();
-    showLocksRequest.setDbname(database);
-    showLocksRequest.setTablename(tableName);
-    ShowLocksResponse response = metaClients.run(client -> client.showLocks(showLocksRequest));
-    for (ShowLocksResponseElement lock : response.getLocks()) {
-      if (lock.getAgentInfo().equals(agentInfo)) {
-        // We found our lock
-        return new LockInfo(lock.getLockid(), lock.getState());
-      }
-    }
-
-    // Not found anything
-    return null;
-  }
-
-  private static class HiveLockHeartbeat implements Runnable {
-    private final ClientPool<IMetaStoreClient, TException> hmsClients;
-    private final long lockId;
-    private final long intervalMs;
-    private ScheduledFuture<?> future;
-    private volatile Exception encounteredException = null;
-
-    HiveLockHeartbeat(
-        ClientPool<IMetaStoreClient, TException> hmsClients, long lockId, long intervalMs) {
-      this.hmsClients = hmsClients;
-      this.lockId = lockId;
-      this.intervalMs = intervalMs;
-      this.future = null;
-    }
-
-    @Override
-    public void run() {
-      try {
-        hmsClients.run(
-            client -> {
-              client.heartbeat(0, lockId);
-              return null;
-            });
-      } catch (TException | InterruptedException e) {
-        this.encounteredException = e;
-        throw new CommitFailedException(e, "Failed to heartbeat for lock: %d", lockId);
-      }
-    }
-
-    public void schedule(ScheduledExecutorService scheduler) {
-      future =
-          scheduler.scheduleAtFixedRate(this, intervalMs / 2, intervalMs, TimeUnit.MILLISECONDS);
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {
+    if (metadata.properties().get(TableProperties.LOCK_HIVE_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.LOCK_HIVE_ENABLED, false);
     }
 
-    public void cancel() {
-      if (future != null) {
-        future.cancel(false);
-      }
-    }
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.LOCK_HIVE_ENABLED_DEFAULT);
   }
 
-  private static class LockInfo {
-    private long lockId;
-    private LockState lockState;
-
-    private LockInfo() {
-      this.lockId = -1;
-      this.lockState = null;
-    }
-
-    private LockInfo(long lockId, LockState lockState) {
-      this.lockId = lockId;
-      this.lockState = lockState;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("lockId", lockId)
-          .add("lockState", lockState)
-          .toString();
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {

Review Comment:
   Could we delegate to LockManager to decide what locks need to be created (create an abstraction)? I think it would be more convenient to work in the future. WDYT?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1083871753


##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String LOCK_HIVE_ENABLED = "lock.hive.enabled";

Review Comment:
   > Thanks for these efforts @pvary . I left some comments. I think it would be easier to review if we just did the refactor first, then the changes are more minimal for supporting the new NoLock mechanism, which I havent wrapped my mind around fully yet.
   
   Thanks for the review!
   Separated the lock refactor to a different PR: #6648, and after it is merged, we can go through the `NoLock` change



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1083879567


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock

Review Comment:
   Thanks for the comment - did the appropriate changes in #6648. Please check there



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
+   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
+   * otherwise an appropriate exception is thrown.
+   *
+   * @return The created lock
+   * @throws LockException When we are not able to fill the hostname for lock creation, or there is
+   *     an error during lock creation
+   */
+  @SuppressWarnings("ReverseDnsLookup")
+  private LockInfo tryLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    final LockComponent lockComponent =

Review Comment:
   Removed



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1083901709


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
+   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
+   * otherwise an appropriate exception is thrown.
+   *
+   * @return The created lock
+   * @throws LockException When we are not able to fill the hostname for lock creation, or there is
+   *     an error during lock creation
+   */
+  @SuppressWarnings("ReverseDnsLookup")
+  private LockInfo tryLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    final LockComponent lockComponent =
+        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    final LockRequest lockRequest =
+        new LockRequest(
+            Lists.newArrayList(lockComponent), System.getProperty("user.name"), hostName);
+
+    // Only works in Hive 2 or later.
+    if (HiveVersion.min(HiveVersion.HIVE_2)) {
+      lockRequest.setAgentInfo(agentInfo);
+    }
+
+    AtomicBoolean interrupted = new AtomicBoolean(false);
+    Tasks.foreach(lockRequest)
+        .retry(Integer.MAX_VALUE - 100)
+        .exponentialBackoff(
+            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
+        .shouldRetryTest(e -> !interrupted.get() && HiveVersion.min(HiveVersion.HIVE_2))

Review Comment:
   `HiveTableOperations.doCommit` used to handle `TException`, `InterruptedException`, `UnknownHostException` before the change by converting them to a `RuntimeException`. After the change we throw a `CommitFailedException` which is a `RuntimeException`.
   
   It is definitely some change in the API, but I think the `CommitFailedException` better describes the situation than a random `RuntimeException`.
   
   TBH I would change the remaining 2 places where we throw `RuntimeException`s (wrapping `TException` and `InterruptedException`) from the `loadHmsTable`, but we have specific tests for these, so I decided against it.
   
   We can continue the discussion on #6648



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1073155042


##########
docs/configuration.md:
##########
@@ -175,8 +175,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.lock.hive.enabled                 | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  | 
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.lock.hive.enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)

Review Comment:
   I have backported the changes to all of the activitie Hive branches and all of the PRs have been merged:
   - master/HIVE-26882: https://github.com/apache/hive/pull/3888
   - branch-3/HIVE-26882: https://github.com/apache/hive/pull/3943
   - branch-3.1/HIVE-26882: : https://github.com/apache/hive/pull/3944
   - branch-2/HIVE-26882: : https://github.com/apache/hive/pull/3946
   - branch-2.1/HIVE-26882: : https://github.com/apache/hive/pull/3947
   



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1161305556


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   It is.
   So once it get released, we can check the version. 



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1083877116


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -53,9 +55,23 @@ private MetastoreUtil() {}
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If possible, an environmental context will

Review Comment:
   I am not entirely sure I understand this comment. Are you suggesting to move the `If possible, an environmental context will...` part of the class comment to an inline comment?
   
   I would like to keep this as a class comment as this is would be part of the javadoc for this method, and as a user of the method I would definitely like to know if to data I provided for the new table is changed by the method itself before submitting it to the HMS.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1174880114


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Created #7418



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1176099905


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Checked, and every Hive 2 should have the required method on the HMS API:
   https://github.com/apache/hive/blob/release-2.0.0/metastore/if/hive_metastore.thrift
   
   So we are good here



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1161328364


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   My thinking was somewhat different. 
   I expect that we should highlight in the documentation the possible issues, and then with the configuration provide flexibility to use it correctly.
   
   I expect that if someone does the effort of backporting the required changes, then they have one of the following:
   1. They control everything and migrate with a big boom
   2. Few problematic tables which they need to write without locks, but they do not want to touch the other tables.
   3. They have a few tables which are written by old clients and most of the tables will be written by new clients
   
   In the case of 2/3 there might be a situation where a particular client would like to write one table with locks and one table without locks, but there is a general rule which most of the clients would like to adhere.
   
   Do I overcomplicate things? If we do not expect this kind of situation, then I would prefer a simpler/table level conf.
   
   What do you think?
   Thanks, Peter 



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1161305176


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -263,6 +266,15 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
         throw e;
 
       } catch (Throwable e) {
+        if (e.getMessage()

Review Comment:
   That is specifically set as a possible returned exception already on the API. About 2 years ago I have proposed to clean up the exception handling by adding/removing exceptions to the methods, but my suggestion got rejected as it would be a breaking change.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1160085184


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Actually  I was thinking more about it, even if we were to make this check return only if both are NO_LOCK, it still doesnt address the original concern that if one writer does not have NO_LOCK  but the table has NO_LOCK, then that writer will not switch to NO_LOCK and has chance to be corrupted by others that do.
   
   So I see two options:
   
   - all writers just follow table's NO_LOCK config
   - writer could do a check, if table is set to NO_LOCK but they are not set to NO_LOCK, throw an exception and prevent the write.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1160044735


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Yea I was thinking, can we throw a ValidationException here then, if the API is to pass in env context, and the alter_table method that we load does not take it?



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159171448


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Why do we want to override the table property using conf?  I thought all writes should use the hive lock enabled, or all writers should not.



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.

Review Comment:
   extra 'with'



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -263,6 +266,15 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
         throw e;
 
       } catch (Throwable e) {
+        if (e.getMessage()

Review Comment:
   This is a bit unfortunate, we didn't go with a proper subclass of MetaException



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Also, is there any way to validate Hive compaitibility, to prevent old Hive version from disabling Hive locks?



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental

Review Comment:
   How about "If the HMS supports then" -> "If the HMS supports it"



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Maybe to validate hive version, one way is  check here if alter_table method has enough arguments for the env?  If not, I feel the whole scheme is not going to work.



##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String HIVE_LOCK_ENABLED = "hive.lock.enabled";
+  public static final boolean HIVE_LOCK_ENABLED_DEFAULT = true;

Review Comment:
   Interesting, I  always read  'engine' as for hive-mr, ie running hive-on-iceberg, whereas this is for all engines (spark/flink) that use hive catalog.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159182809


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental

Review Comment:
   How about "If the HMS supports then" -> "If the HMS supports it".  I think the former works more in Java and not english :)



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1169477396


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   The "env"-less alter table will only be used with old HMS clients so we do not have issues with adding env variables to the HMS call which does not handle it.
   
   We still might provide env values which the server does not act upon (I was talking about this case in my previous comments here), but at least we can be sure that we do not break the API calls



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1171970109


##########
docs/configuration.md:
##########
@@ -178,8 +178,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.engine.hive.lock-enabled          | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  |
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.engine.hive.lock-enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
+is available on the Hive Metastore server and every writer of a given table is using Iceberg 1.2 or later.

Review Comment:
   We should change this to warn.  We also miss warning about some catalogs setting true and others setting false.  
   
   How about this to add some details.  I use HiveCatalog here, even though its not defined in the doc, otherwise its quite lengthy to continue the language- "catalog using Hive Metastore connector"
   
   
   Warn: Setting `iceberg.engine.hive.lock-enabled`=`false` will cause HiveCatalog to commit to tables without using Hive locks.  This should only be set to `false` if all following conditions are met: 
   * [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
   is available on the Hive Metastore server
   * All other HiveCatalogs committing to tables that this HiveCatalog commits to are also on Iceberg 1.3 or later
   * All other HiveCatalogs committing to tables that this HiveCatalog commits to have also disabled Hive locks on commit.
   
   Failing to ensure these conditions risks corrupting the table.
   
   Even with `iceberg.engine.hive.lock-enabled` set to `false`, a HiveCatalog can still use locks for individual tables by setting the table property 'engine.hive.lock-enabled'='true'.  This is useful in the case where other HiveCatalogs cannot be upgraded and set to commit without using Hive locks.
   



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1161989553


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Oh maybe I was not clear, I meant the alter_method that we load dynamically:  https://github.com/apache/iceberg/blob/master/hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java#L29  .  Original question was just asking if our method object is the one loaded without env, and we pass in env here, what will happen. (is error message good enough)



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1161329132


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   We can not distinguish between the 2 cases. The API is backwards compatible (as expected by the Hive team), so the HMS will always take the env context input variables, but will not act on them.
   
   I think this is not that interesting case, as the writer should have a clear info about the HMS they are connecting to.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1161989553


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Oh maybe I was not clear, I meant the alter_method that we load dynamically:  https://github.com/apache/iceberg/blob/master/hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java#L29  .  Original question was just asking: if our method object is the one loaded without env, and we pass in env here, what will happen? (is the error message good enough for user to decipher, and it wont silently pass?)  Its a pretty rare scenario, just wondering.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1174181304


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Hey sorry , I thought I commented on this, but must have missed it.  I was thinking something like:
   
   if (env.size() > 0) {
     Preconditions.checkArgument("Environment context is non-empty but alter_table method does not support it",  ALTER_TABLE.args().size() == 5);
   }
   
   What do you think?  Isn't alter_table only called with envContext is the new code?



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1174178671


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Let's make an issue to track it on Iceberg side



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1171929442


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   OK I think I get the difference as well, if I can summarize, we have two choices
   
   1.  catalog config = NO_LOCK => start writing to all tables with no_lock, unless table is set explicitly to LOCK.
   2.  catalog config = NO_LOCK => start writing to tables with no_lock only if table set explicitly to NO_LOCK.
   
   In both cases, we can achieve case 2/3, the issue is whether the user has to set the tables explicitly to exclude or include them in new system.
   
   You are right that maybe Option 2 is too much of a burden for the user to manually mark all tables as NO_LOCK.  If all Iceberg clients are upgraded in the system, Option 2 may be a good check as then we are sure all of them will write NO_LOCK together, but we come back to the same problem if some clients are not upgraded.  (which is what case 2/3 is trying to solve).
   
   Let me think about it, but I think you are right that we cannot do better in the mixed-version scenarios.



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


[GitHub] [iceberg] haizhou-zhao commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
haizhou-zhao commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1072851109


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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 com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetastoreLock implements HiveLock {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreLock.class);
+  private static final String HIVE_ACQUIRE_LOCK_TIMEOUT_MS = "iceberg.hive.lock-timeout-ms";
+  private static final String HIVE_LOCK_CHECK_MIN_WAIT_MS = "iceberg.hive.lock-check-min-wait-ms";
+  private static final String HIVE_LOCK_CHECK_MAX_WAIT_MS = "iceberg.hive.lock-check-max-wait-ms";
+  private static final String HIVE_LOCK_CREATION_TIMEOUT_MS =
+      "iceberg.hive.lock-creation-timeout-ms";
+  private static final String HIVE_LOCK_CREATION_MIN_WAIT_MS =
+      "iceberg.hive.lock-creation-min-wait-ms";
+  private static final String HIVE_LOCK_CREATION_MAX_WAIT_MS =
+      "iceberg.hive.lock-creation-max-wait-ms";
+  private static final String HIVE_LOCK_HEARTBEAT_INTERVAL_MS =
+      "iceberg.hive.lock-heartbeat-interval-ms";
+  private static final String HIVE_TABLE_LEVEL_LOCK_EVICT_MS =
+      "iceberg.hive.table-level-lock-evict-ms";
+
+  private static final long HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT = 3 * 60 * 1000; // 3 minutes
+  private static final long HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT = 50; // 50 milliseconds
+  private static final long HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT = 5 * 1000; // 5 seconds
+  private static final long HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = 4 * 60 * 1000; // 4 minutes
+  private static final long HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(10);
+
+  private final ClientPool<IMetaStoreClient, TException> metaClients;
+  private final String databaseName;
+  private final String tableName;
+  private final String fullName;
+  private final long lockAcquireTimeout;
+  private final long lockCheckMinWaitTime;
+  private final long lockCheckMaxWaitTime;
+  private final long lockCreationTimeout;
+  private final long lockCreationMinWaitTime;
+  private final long lockCreationMaxWaitTime;
+  private final long lockHeartbeatIntervalTime;
+  private final ScheduledExecutorService exitingScheduledExecutorService;
+
+  private final String agentInfo;
+
+  private static Cache<String, ReentrantLock> commitLockCache;
+
+  private Optional<Long> hmsLockId = Optional.empty();
+  private Optional<ReentrantLock> jvmLock = Optional.empty();
+  private HiveLockHeartbeat hiveLockHeartbeat = null;
+
+  public MetastoreLock(
+      Configuration conf,
+      ClientPool<IMetaStoreClient, TException> metaClients,
+      String catalogName,
+      String databaseName,
+      String tableName) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+
+    this.lockAcquireTimeout =
+        conf.getLong(HIVE_ACQUIRE_LOCK_TIMEOUT_MS, HIVE_ACQUIRE_LOCK_TIMEOUT_MS_DEFAULT);
+    this.lockCheckMinWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MIN_WAIT_MS, HIVE_LOCK_CHECK_MIN_WAIT_MS_DEFAULT);
+    this.lockCheckMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CHECK_MAX_WAIT_MS, HIVE_LOCK_CHECK_MAX_WAIT_MS_DEFAULT);
+    this.lockCreationTimeout =
+        conf.getLong(HIVE_LOCK_CREATION_TIMEOUT_MS, HIVE_LOCK_CREATION_TIMEOUT_MS_DEFAULT);
+    this.lockCreationMinWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MIN_WAIT_MS, HIVE_LOCK_CREATION_MIN_WAIT_MS_DEFAULT);
+    this.lockCreationMaxWaitTime =
+        conf.getLong(HIVE_LOCK_CREATION_MAX_WAIT_MS, HIVE_LOCK_CREATION_MAX_WAIT_MS_DEFAULT);
+    this.lockHeartbeatIntervalTime =
+        conf.getLong(HIVE_LOCK_HEARTBEAT_INTERVAL_MS, HIVE_LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    long tableLevelLockCacheEvictionTimeout =
+        conf.getLong(HIVE_TABLE_LEVEL_LOCK_EVICT_MS, HIVE_TABLE_LEVEL_LOCK_EVICT_MS_DEFAULT);
+
+    this.agentInfo = "Iceberg-" + UUID.randomUUID();
+
+    this.exitingScheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("iceberg-hive-lock-heartbeat-" + fullName + "-%d")
+                .build());
+
+    initTableLevelLockCache(tableLevelLockCacheEvictionTimeout);
+  }
+
+  @Override
+  public void lock() throws LockException {
+    // getting a process-level lock per table to avoid concurrent commit attempts to the same table
+    // from the same
+    // JVM process, which would result in unnecessary and costly HMS lock acquisition requests
+    acquireJvmLock();
+
+    // Getting HMS lock
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    hiveLockHeartbeat =
+        new HiveLockHeartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    hiveLockHeartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (hiveLockHeartbeat.encounteredException != null) {
+      throw new LockException(
+          hiveLockHeartbeat.encounteredException,
+          "Failed to heartbeat for hive lock. %s",
+          hiveLockHeartbeat.encounteredException.getMessage());
+    }
+    if (hiveLockHeartbeat == null
+        || hiveLockHeartbeat.future == null
+        || hiveLockHeartbeat.future.isCancelled()) {
+      throw new LockException("Lock is not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (hiveLockHeartbeat != null) {
+      hiveLockHeartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = tryLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException error = null;
+
+    try {
+      if (lockInfo.lockState.equals(LockState.WAITING)) {
+        // Retry count is the typical "upper bound of retries" for Tasks.run() function. In fact,
+        // the maximum number of
+        // attempts the Tasks.run() would try is `retries + 1`. Here, for checking locks, we use
+        // timeout as the
+        // upper bound of retries. So it is just reasonable to set a large retry count. However, if
+        // we set
+        // Integer.MAX_VALUE, the above logic of `retries + 1` would overflow into
+        // Integer.MIN_VALUE. Hence,
+        // the retry is set conservatively as `Integer.MAX_VALUE - 100` so it doesn't hit any
+        // boundary issues.
+        Tasks.foreach(lockInfo.lockId)
+            .retry(Integer.MAX_VALUE - 100)
+            .exponentialBackoff(lockCheckMinWaitTime, lockCheckMaxWaitTime, lockAcquireTimeout, 1.5)
+            .throwFailureWhenFinished()
+            .onlyRetryOn(WaitingForLockException.class)
+            .run(
+                id -> {
+                  try {
+                    LockResponse response = metaClients.run(client -> client.checkLock(id));
+                    LockState newState = response.getState();
+                    lockInfo.lockState = newState;
+                    if (newState.equals(LockState.WAITING)) {
+                      throw new WaitingForLockException(
+                          String.format(
+                              "Waiting for lock on table %s.%s", databaseName, tableName));
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.interrupted(); // Clear the interrupt status flag
+                    LOG.warn(
+                        "Interrupted while waiting for lock on table {}.{}",
+                        databaseName,
+                        tableName,
+                        e);
+                  }
+                },
+                TException.class);
+      }
+    } catch (WaitingForLockException e) {
+      timeout = true;
+      duration = System.currentTimeMillis() - start;
+    } catch (TException e) {
+      error = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      return lockInfo.lockId;
+    }
+
+    // timeout and do not have lock acquired
+    if (timeout) {
+      throw new LockException(
+          "Timed out after %s ms waiting for lock on %s.%s", duration, databaseName, tableName);
+    }
+
+    // On thrift error and do not have lock acquired
+    if (error != null) {
+      throw new LockException(
+          error, "Metastore operation failed for %s.%s", databaseName, tableName);
+    }
+
+    // Just for safety. We should not get here.
+    throw new LockException(
+        "Could not acquire the lock on %s.%s, lock request ended in state %s",
+        databaseName, tableName, lockInfo.lockState);
+  }
+
+  /**
+   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
+   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
+   * otherwise an appropriate exception is thrown.
+   *
+   * @return The created lock
+   * @throws LockException When we are not able to fill the hostname for lock creation, or there is
+   *     an error during lock creation
+   */
+  @SuppressWarnings("ReverseDnsLookup")
+  private LockInfo tryLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    final LockComponent lockComponent =
+        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    final LockRequest lockRequest =
+        new LockRequest(
+            Lists.newArrayList(lockComponent), System.getProperty("user.name"), hostName);

Review Comment:
   Just FYI, I'm trying to promote the usage of Hadoop's UserGroupInformation instead of OS user when interacting with HMS. In production Hive environment where Hadoop authN mechanism is enabled, the Hive ecosystem will not integrate well with the local OS user name (they typically use Kerberos principal)
   
   Details at: https://github.com/apache/iceberg/pull/6324/files#diff-4f691cd4a9db7de8859c76a837a56fd507137d5a1570a0cfed3db6fca41c0f19R32



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


[GitHub] [iceberg] deniskuzZ commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1071409985


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -809,168 +603,38 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
   }
 
   /**
-   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
-   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
-   * otherwise an appropriate exception is thrown.
+   * Returns if the hive locking should be enabled on the table, or not.
    *
-   * @param agentInfo The agentInfo which should be used during lock creation
-   * @return The created lock
-   * @throws UnknownHostException When we are not able to fill the hostname for lock creation
-   * @throws TException When there is an error during lock creation
-   */
-  @SuppressWarnings("ReverseDnsLookup")
-  private LockInfo tryLock(String agentInfo) throws UnknownHostException, TException {
-    LockInfo lockInfo = new LockInfo();
-
-    final LockComponent lockComponent =
-        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, database);
-    lockComponent.setTablename(tableName);
-    final LockRequest lockRequest =
-        new LockRequest(
-            Lists.newArrayList(lockComponent),
-            System.getProperty("user.name"),
-            InetAddress.getLocalHost().getHostName());
-
-    // Only works in Hive 2 or later.
-    if (HiveVersion.min(HiveVersion.HIVE_2)) {
-      lockRequest.setAgentInfo(agentInfo);
-    }
-
-    Tasks.foreach(lockRequest)
-        .retry(Integer.MAX_VALUE - 100)
-        .exponentialBackoff(
-            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
-        .shouldRetryTest(e -> e instanceof TException && HiveVersion.min(HiveVersion.HIVE_2))
-        .throwFailureWhenFinished()
-        .run(
-            request -> {
-              try {
-                LockResponse lockResponse = metaClients.run(client -> client.lock(request));
-                lockInfo.lockId = lockResponse.getLockid();
-                lockInfo.lockState = lockResponse.getState();
-              } catch (TException te) {
-                LOG.warn("Failed to acquire lock {}", request, te);
-                try {
-                  // If we can not check for lock, or we do not find it, then rethrow the exception
-                  // Otherwise we are happy as the findLock sets the lockId and the state correctly
-                  if (!HiveVersion.min(HiveVersion.HIVE_2)) {
-                    LockInfo lockFound = findLock(agentInfo);
-                    if (lockFound != null) {
-                      lockInfo.lockId = lockFound.lockId;
-                      lockInfo.lockState = lockFound.lockState;
-                      LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
-                      return;
-                    }
-                  }
-
-                  throw te;
-                } catch (InterruptedException e) {
-                  Thread.currentThread().interrupt();
-                  LOG.warn(
-                      "Interrupted while checking for lock on table {}.{}", database, tableName, e);
-                  throw new RuntimeException("Interrupted while checking for lock", e);
-                }
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                LOG.warn("Interrupted while acquiring lock on table {}.{}", database, tableName, e);
-                throw new RuntimeException("Interrupted while acquiring lock", e);
-              }
-            },
-            TException.class);
-
-    // This should be initialized always, or exception should be thrown.
-    LOG.debug("Lock {} created for table {}.{}", lockInfo, database, tableName);
-    return lockInfo;
-  }
-
-  /**
-   * Search for the locks using HMSClient.showLocks identified by the agentInfo. If the lock is
-   * there, then a {@link LockInfo} object is returned. If the lock is not found <code>null</code>
-   * is returned.
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#LOCK_HIVE_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#LOCK_HIVE_ENABLED_DEFAULT}
+   * </ol>
    *
-   * @param agentInfo The key for searching the locks
-   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
    */
-  private LockInfo findLock(String agentInfo) throws TException, InterruptedException {
-    Preconditions.checkArgument(
-        HiveVersion.min(HiveVersion.HIVE_2),
-        "Minimally Hive 2 HMS client is needed to find the Lock using the showLocks API call");
-    ShowLocksRequest showLocksRequest = new ShowLocksRequest();
-    showLocksRequest.setDbname(database);
-    showLocksRequest.setTablename(tableName);
-    ShowLocksResponse response = metaClients.run(client -> client.showLocks(showLocksRequest));
-    for (ShowLocksResponseElement lock : response.getLocks()) {
-      if (lock.getAgentInfo().equals(agentInfo)) {
-        // We found our lock
-        return new LockInfo(lock.getLockid(), lock.getState());
-      }
-    }
-
-    // Not found anything
-    return null;
-  }
-
-  private static class HiveLockHeartbeat implements Runnable {
-    private final ClientPool<IMetaStoreClient, TException> hmsClients;
-    private final long lockId;
-    private final long intervalMs;
-    private ScheduledFuture<?> future;
-    private volatile Exception encounteredException = null;
-
-    HiveLockHeartbeat(
-        ClientPool<IMetaStoreClient, TException> hmsClients, long lockId, long intervalMs) {
-      this.hmsClients = hmsClients;
-      this.lockId = lockId;
-      this.intervalMs = intervalMs;
-      this.future = null;
-    }
-
-    @Override
-    public void run() {
-      try {
-        hmsClients.run(
-            client -> {
-              client.heartbeat(0, lockId);
-              return null;
-            });
-      } catch (TException | InterruptedException e) {
-        this.encounteredException = e;
-        throw new CommitFailedException(e, "Failed to heartbeat for lock: %d", lockId);
-      }
-    }
-
-    public void schedule(ScheduledExecutorService scheduler) {
-      future =
-          scheduler.scheduleAtFixedRate(this, intervalMs / 2, intervalMs, TimeUnit.MILLISECONDS);
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {
+    if (metadata.properties().get(TableProperties.LOCK_HIVE_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.LOCK_HIVE_ENABLED, false);
     }
 
-    public void cancel() {
-      if (future != null) {
-        future.cancel(false);
-      }
-    }
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.LOCK_HIVE_ENABLED_DEFAULT);
   }
 
-  private static class LockInfo {
-    private long lockId;
-    private LockState lockState;
-
-    private LockInfo() {
-      this.lockId = -1;
-      this.lockState = null;
-    }
-
-    private LockInfo(long lockId, LockState lockState) {
-      this.lockId = lockId;
-      this.lockState = lockState;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("lockId", lockId)
-          .add("lockState", lockState)
-          .toString();
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName);

Review Comment:
   prev locking strategy migrated to MetastoreLock, right?



##########
docs/configuration.md:
##########
@@ -175,8 +175,13 @@ The HMS table locking is a 2-step process:
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.lock.hive.enabled                 | true            | If enabled HMS locks will be used to ensure of the atomicity of the commits  | 
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Note: `iceberg.lock.hive.enabled` should only be set to `false` if [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)

Review Comment:
   Could it cause some issues in the case of HA and not consistent HMS configs across the instances? 



##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String LOCK_HIVE_ENABLED = "lock.hive.enabled";

Review Comment:
   minor, why not `hive.lock.enabled`?



##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -809,168 +603,38 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
   }
 
   /**
-   * Tries to create a lock. If the lock creation fails, and it is possible then retries the lock
-   * creation a few times. If the lock creation is successful then a {@link LockInfo} is returned,
-   * otherwise an appropriate exception is thrown.
+   * Returns if the hive locking should be enabled on the table, or not.
    *
-   * @param agentInfo The agentInfo which should be used during lock creation
-   * @return The created lock
-   * @throws UnknownHostException When we are not able to fill the hostname for lock creation
-   * @throws TException When there is an error during lock creation
-   */
-  @SuppressWarnings("ReverseDnsLookup")
-  private LockInfo tryLock(String agentInfo) throws UnknownHostException, TException {
-    LockInfo lockInfo = new LockInfo();
-
-    final LockComponent lockComponent =
-        new LockComponent(LockType.EXCLUSIVE, LockLevel.TABLE, database);
-    lockComponent.setTablename(tableName);
-    final LockRequest lockRequest =
-        new LockRequest(
-            Lists.newArrayList(lockComponent),
-            System.getProperty("user.name"),
-            InetAddress.getLocalHost().getHostName());
-
-    // Only works in Hive 2 or later.
-    if (HiveVersion.min(HiveVersion.HIVE_2)) {
-      lockRequest.setAgentInfo(agentInfo);
-    }
-
-    Tasks.foreach(lockRequest)
-        .retry(Integer.MAX_VALUE - 100)
-        .exponentialBackoff(
-            lockCreationMinWaitTime, lockCreationMaxWaitTime, lockCreationTimeout, 2.0)
-        .shouldRetryTest(e -> e instanceof TException && HiveVersion.min(HiveVersion.HIVE_2))
-        .throwFailureWhenFinished()
-        .run(
-            request -> {
-              try {
-                LockResponse lockResponse = metaClients.run(client -> client.lock(request));
-                lockInfo.lockId = lockResponse.getLockid();
-                lockInfo.lockState = lockResponse.getState();
-              } catch (TException te) {
-                LOG.warn("Failed to acquire lock {}", request, te);
-                try {
-                  // If we can not check for lock, or we do not find it, then rethrow the exception
-                  // Otherwise we are happy as the findLock sets the lockId and the state correctly
-                  if (!HiveVersion.min(HiveVersion.HIVE_2)) {
-                    LockInfo lockFound = findLock(agentInfo);
-                    if (lockFound != null) {
-                      lockInfo.lockId = lockFound.lockId;
-                      lockInfo.lockState = lockFound.lockState;
-                      LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
-                      return;
-                    }
-                  }
-
-                  throw te;
-                } catch (InterruptedException e) {
-                  Thread.currentThread().interrupt();
-                  LOG.warn(
-                      "Interrupted while checking for lock on table {}.{}", database, tableName, e);
-                  throw new RuntimeException("Interrupted while checking for lock", e);
-                }
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                LOG.warn("Interrupted while acquiring lock on table {}.{}", database, tableName, e);
-                throw new RuntimeException("Interrupted while acquiring lock", e);
-              }
-            },
-            TException.class);
-
-    // This should be initialized always, or exception should be thrown.
-    LOG.debug("Lock {} created for table {}.{}", lockInfo, database, tableName);
-    return lockInfo;
-  }
-
-  /**
-   * Search for the locks using HMSClient.showLocks identified by the agentInfo. If the lock is
-   * there, then a {@link LockInfo} object is returned. If the lock is not found <code>null</code>
-   * is returned.
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#LOCK_HIVE_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#LOCK_HIVE_ENABLED_DEFAULT}
+   * </ol>
    *
-   * @param agentInfo The key for searching the locks
-   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
    */
-  private LockInfo findLock(String agentInfo) throws TException, InterruptedException {
-    Preconditions.checkArgument(
-        HiveVersion.min(HiveVersion.HIVE_2),
-        "Minimally Hive 2 HMS client is needed to find the Lock using the showLocks API call");
-    ShowLocksRequest showLocksRequest = new ShowLocksRequest();
-    showLocksRequest.setDbname(database);
-    showLocksRequest.setTablename(tableName);
-    ShowLocksResponse response = metaClients.run(client -> client.showLocks(showLocksRequest));
-    for (ShowLocksResponseElement lock : response.getLocks()) {
-      if (lock.getAgentInfo().equals(agentInfo)) {
-        // We found our lock
-        return new LockInfo(lock.getLockid(), lock.getState());
-      }
-    }
-
-    // Not found anything
-    return null;
-  }
-
-  private static class HiveLockHeartbeat implements Runnable {
-    private final ClientPool<IMetaStoreClient, TException> hmsClients;
-    private final long lockId;
-    private final long intervalMs;
-    private ScheduledFuture<?> future;
-    private volatile Exception encounteredException = null;
-
-    HiveLockHeartbeat(
-        ClientPool<IMetaStoreClient, TException> hmsClients, long lockId, long intervalMs) {
-      this.hmsClients = hmsClients;
-      this.lockId = lockId;
-      this.intervalMs = intervalMs;
-      this.future = null;
-    }
-
-    @Override
-    public void run() {
-      try {
-        hmsClients.run(
-            client -> {
-              client.heartbeat(0, lockId);
-              return null;
-            });
-      } catch (TException | InterruptedException e) {
-        this.encounteredException = e;
-        throw new CommitFailedException(e, "Failed to heartbeat for lock: %d", lockId);
-      }
-    }
-
-    public void schedule(ScheduledExecutorService scheduler) {
-      future =
-          scheduler.scheduleAtFixedRate(this, intervalMs / 2, intervalMs, TimeUnit.MILLISECONDS);
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {
+    if (metadata.properties().get(TableProperties.LOCK_HIVE_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.LOCK_HIVE_ENABLED, false);
     }
 
-    public void cancel() {
-      if (future != null) {
-        future.cancel(false);
-      }
-    }
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.LOCK_HIVE_ENABLED_DEFAULT);
   }
 
-  private static class LockInfo {
-    private long lockId;
-    private LockState lockState;
-
-    private LockInfo() {
-      this.lockId = -1;
-      this.lockState = null;
-    }
-
-    private LockInfo(long lockId, LockState lockState) {
-      this.lockId = lockId;
-      this.lockState = lockState;
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(this)
-          .add("lockId", lockId)
-          .add("lockState", lockState)
-          .toString();
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {

Review Comment:
   Could we delegate to LockManager to decide what locks need to be created? I think it would be more convenient to work in the future. WDYT?



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


[GitHub] [iceberg] haizhou-zhao commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
haizhou-zhao commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1072884107


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/NoLock.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+public class NoLock implements HiveLock {

Review Comment:
   How is this used?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159949811


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   The table property could override the catalog property.
   This way there is a graceful way to enable the config for every writer at the same time:
   - Start by disabling the config on table level
   - Enable the writers one-by-one based on the writers release process
   - When you are sure that every writer uses the new code, then you can change the table config and all of the writers start using the new locking method at the same time



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159183494


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -48,14 +50,28 @@ public class MetastoreUtil {
   private MetastoreUtil() {}
 
   /**
-   * Calls alter_table method using the metastore client. If possible, an environmental context will
-   * be used that turns off stats updates to avoid recursive listing.
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
    */
   public static void alterTable(
       IMetaStoreClient client, String databaseName, String tblName, Table table) {
-    EnvironmentContext envContext =
-        new EnvironmentContext(
-            ImmutableMap.of(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE));
-    ALTER_TABLE.invoke(client, databaseName, tblName, table, envContext);
+    alterTable(client, databaseName, tblName, table, ImmutableMap.of());
+  }
+
+  /**
+   * Calls alter_table method using the metastore client. If the HMS supports then, environmental
+   * context with will be set in a way that turns off stats updates to avoid recursive file listing.
+   */
+  public static void alterTable(
+      IMetaStoreClient client,
+      String databaseName,
+      String tblName,
+      Table table,
+      Map<String, String> extraEnv) {
+    Map<String, String> env = Maps.newHashMapWithExpectedSize(extraEnv.size() + 1);
+    env.putAll(extraEnv);
+    env.put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+

Review Comment:
   Checking, if the version of alter_table does not support env context, it will throw an exception instead of silently fail?



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1159969613


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -263,6 +266,15 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
         throw e;
 
       } catch (Throwable e) {
+        if (e.getMessage()

Review Comment:
   IIRC Thrift does not work well with subclassing. So that was the reason why we do not have proper exception subclasses on HMS API



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1162197847


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Sorry I'm trying but still not understanding how having a client level (catalog level) config, that is fallback to table config, allows case 2/3?  
   
   To me as it is, it seems it causes more potential problem than solves (client says NO_LOCK but table does not have NO_LOCK).  Unless we want to do catalog config as a safety check (throw exception if catalog config is NO_LOCK but table does not have NO_LOCK), as I was mentioning before, which makes more sense to me.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1162198156


##########
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -572,8 +595,39 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
         ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns if the hive locking should be enabled on the table, or not.
+   *
+   * <p>The decision is made like this:
+   *
+   * <ol>
+   *   <li>Table property value {@link TableProperties#HIVE_LOCK_ENABLED}
+   *   <li>If the table property is not set then check the hive-site.xml property value {@link
+   *       ConfigProperties#LOCK_HIVE_ENABLED}
+   *   <li>If none of the above is enabled then use the default value {@link
+   *       TableProperties#HIVE_LOCK_ENABLED_DEFAULT}
+   * </ol>
+   *
+   * @param metadata Table metadata to use
+   * @param conf The hive configuration to use
+   * @return if the hive engine related values should be enabled or not
+   */
+  private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) {

Review Comment:
   Great, I think this would be very important to get in for users of OSS Hive, its a bit hacky otherwise.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1143949780


##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String HIVE_LOCK_ENABLED = "hive.lock.enabled";
+  public static final boolean HIVE_LOCK_ENABLED_DEFAULT = true;

Review Comment:
   Could we keep the `engine.hive` prefix? How about `engine.hive.lock-enabled`?



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1143950357


##########
core/src/main/java/org/apache/iceberg/hadoop/ConfigProperties.java:
##########
@@ -23,5 +23,6 @@ public class ConfigProperties {
   private ConfigProperties() {}
 
   public static final String ENGINE_HIVE_ENABLED = "iceberg.engine.hive.enabled";
+  public static final String LOCK_HIVE_ENABLED = "iceberg.lock.hive.enabled";

Review Comment:
   This should follow the convention from table properties, like `engine.hive.enabled`, adding the `iceberg` prefix to the table property.



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


[GitHub] [iceberg] pvary commented on a diff in pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#discussion_r1071910972


##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -303,6 +303,9 @@ private TableProperties() {}
   public static final String ENGINE_HIVE_ENABLED = "engine.hive.enabled";
   public static final boolean ENGINE_HIVE_ENABLED_DEFAULT = false;
 
+  public static final String LOCK_HIVE_ENABLED = "lock.hive.enabled";

Review Comment:
   I used the same pattern than with `engine.hive.enabled`, but if we have a consensus I am happy to use whatever config name is decided upon 😄 



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


[GitHub] [iceberg] pvary commented on pull request #6570: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882

Posted by GitBox <gi...@apache.org>.
pvary commented on PR #6570:
URL: https://github.com/apache/iceberg/pull/6570#issuecomment-1385051268

   > LGTM, thanks for the path Peter, that should definitely improve the Hive iceberg performance. Let me know what kind of help is needed from the Hive folks
   
   Hi @deniskuzZ, good to hear from you. I hope you are ok considering...
   
   Happily, I got the approvals from @ayushtkn which were needed for backporting HIVE-26882 to Hive release branches. So the only thing remaining on Hive side is to have a release having the code available, so other projects could start using it.
   
   Thanks for the review!
   Peter


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