You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "InvisibleProgrammer (via GitHub)" <gi...@apache.org> on 2023/05/04 14:46:46 UTC

[GitHub] [hive] InvisibleProgrammer opened a new pull request, #4291: Hive-27306: port iceberg catalog changes

InvisibleProgrammer opened a new pull request, #4291:
URL: https://github.com/apache/hive/pull/4291

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235851050


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java:
##########
@@ -332,10 +338,13 @@ public void testLoadCatalogLocation() {
   @Test
   public void testLoadCatalogUnknown() {
     String catalogName = "barCatalog";
-    conf.set(InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE), "fooType");
-    AssertHelpers.assertThrows(
-        "should complain about catalog not supported", UnsupportedOperationException.class,
-        "Unknown catalog type:", () -> Catalogs.loadCatalog(conf, catalogName));
+    conf.set(
+        InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE),
+        "fooType");

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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235030041


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -176,11 +180,10 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
     CommitStatus commitStatus = CommitStatus.FAILURE;
     boolean updateHiveTable = false;
-    HiveCommitLock commitLock = null;
 
+    HiveLock lock = lockObject(metadata);

Review Comment:
   are we missing `Core: Avoid creating new metadata file on registerTable (#6591)`here? 
   ````
   boolean newTable = base == null;
       String newMetadataLocation = writeNewMetadataIfRequired(newTable, metadata);
   ````



##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -176,11 +180,10 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
     CommitStatus commitStatus = CommitStatus.FAILURE;
     boolean updateHiveTable = false;
-    HiveCommitLock commitLock = null;
 
+    HiveLock lock = lockObject(metadata);

Review Comment:
   are we missing `Core: Avoid creating new metadata file on registerTable (#6591)`here? 
   ````
   boolean newTable = base == null;
   String newMetadataLocation = writeNewMetadataIfRequired(newTable, metadata);
   ````



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235137523


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java:
##########
@@ -21,25 +21,61 @@
 
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
 import org.apache.thrift.TException;
+import org.immutables.value.Value;
 
+/**
+ * A ClientPool that caches the underlying HiveClientPool instances.
+ *
+ * <p>The following key elements are supported and can be specified via {@link
+ * CatalogProperties#CLIENT_POOL_CACHE_KEYS}:
+ *
+ * <ul>
+ *   <li>ugi - the Hadoop UserGroupInformation instance that represents the current user using the
+ *       cache.
+ *   <li>user_name - similar to UGI but only includes the user's name determined by
+ *       UserGroupInformation#getUserName.
+ *   <li>conf - name of an arbitrary configuration. The value of the configuration will be extracted
+ *       from catalog properties and added to the cache key. A conf element should start with a
+ *       "conf:" prefix which is followed by the configuration name. E.g. specifying "conf:a.b.c"
+ *       will add "a.b.c" to the key, and so that configurations with different default catalog
+ *       wouldn't share the same client pool. Multiple conf elements can be specified.
+ * </ul>
+ */
 public class CachedClientPool implements ClientPool<IMetaStoreClient, TException> {
 
-  private static Cache<String, HiveClientPool> clientPoolCache;
+  private static final String CONF_ELEMENT_PREFIX = "conf:";
+
+  private static Cache<Key, HiveClientPool> clientPoolCache;
 
   private final Configuration conf;
   private final String metastoreUri;

Review Comment:
   Are we talking about the `CONF_ELEMENT_PREFIX` or the `clientPoolCache` fields. The element prefix is used in the `extractKey` method. And the cache is  used via the `run` methods. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235188314


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -19,57 +19,61 @@
 
 package org.apache.iceberg.hive;
 
+import java.util.Map;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.iceberg.common.DynMethods;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 
 public class MetastoreUtil {
-
-  // this class is unique to Hive3 and cannot be found in Hive2, therefore a good proxy to see if
-  // we are working against Hive3 dependencies
-  private static final String HIVE3_UNIQUE_CLASS = "org.apache.hadoop.hive.serde2.io.DateWritableV2";
-
-  private static final DynMethods.UnboundMethod ALTER_TABLE = DynMethods.builder("alter_table")
-      .impl(IMetaStoreClient.class, "alter_table_with_environmentContext",
-          String.class, String.class, Table.class, EnvironmentContext.class)
-      .impl(IMetaStoreClient.class, "alter_table",
-          String.class, String.class, Table.class, EnvironmentContext.class)
-      .impl(IMetaStoreClient.class, "alter_table",
-          String.class, String.class, Table.class)
+  private static final DynMethods.UnboundMethod ALTER_TABLE =
+      DynMethods.builder("alter_table")
+          .impl(
+              IMetaStoreClient.class,
+              "alter_table_with_environmentContext",
+              String.class,
+              String.class,
+              Table.class,
+              EnvironmentContext.class)
+          .impl(
+              IMetaStoreClient.class,
+              "alter_table",
+              String.class,
+              String.class,
+              Table.class,
+              EnvironmentContext.class)
+          .impl(IMetaStoreClient.class, "alter_table", String.class, String.class, Table.class)
       .build();
 
-  private static final boolean HIVE3_PRESENT_ON_CLASSPATH = detectHive3();

Review Comment:
   sorry, my bad



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ayushtkn commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "ayushtkn (via GitHub)" <gi...@apache.org>.
ayushtkn commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235191176


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java:
##########
@@ -205,6 +205,11 @@ public void testResiduals() throws Exception {
 
   @Test
   public void testFailedResidualFiltering() throws Exception {
+    // Vectorization is not yet supported for AVRO
+    if (this.fileFormat.equals(FileFormat.AVRO)) {

Review Comment:
   Use ``Assume.assumeTrue``



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235311659


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -510,6 +548,53 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
       return metadata.propertyAsBoolean(TableProperties.ENGINE_HIVE_ENABLED, false);
     }
 
-    return conf.getBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
+    return conf.getBoolean(
+        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) {
+    if (metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.HIVE_LOCK_ENABLED, false);
+    }
+
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.HIVE_LOCK_ENABLED_DEFAULT);
+  }
+
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      Optional<Long> txnId = Optional.empty();
+
+      SessionState sessionState = SessionState.get();
+      if (sessionState != null) {
+        HiveTxnManager txnMgr = sessionState.getTxnMgr();
+        if (txnMgr != null) {
+          txnId = Optional.of(txnMgr.getCurrentTxnId());
+        }
+      }
+
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName,   txnId);

Review Comment:
   Yes, that is the reason.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236780270


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   No. The original plan was to upgrade to 1.2.1 and additionally, some extra commits. As the ticket says: https://issues.apache.org/jira/browse/HIVE-27306
   
   ```
   Extra that we have to take care of:
   
   c3232b664745ebf761b6a74f4c5b55cc48bfd209: Hive: Use EnvironmentContext instead of Hive Locks to provide transactional commits after HIVE-26882;
   ```
   
   So that commit was after 1.2.1 release but before 1.3.0. That means, when I ported it to Hive, I had to introduce those files in the patches to make this commit working. As you can see in the Iceberg commit (https://github.com/apache/iceberg/commit/c3232b664745ebf761b6a74f4c5b55cc48bfd209), those files are in the core module and it is not part of the Hive-Iceberg integration. But as at that time Iceberg dependency was 1.2.1, that was the way to see those changes in Iceberg. 
   
   Right after I finished with the 1.2.1, 1.3.0 released. And it contained only few minor changes that haven't ported yet so I decided to port the remaining part in this PR. That is why this PR is not about upgrading from 1.1.0 to 1.2.1 but to 1.3.0. 
   
   But, as 1.3.0 dependency contains those changes, with upgrading Iceberg dependency to 1.3.0, they could be removed (https://github.com/apache/hive/pull/4291/commits/ddba7deeb42ba8974e1cd49be96052da3ea259f4).
   
   That is the reason why this PR doesn't contain those files. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236958734


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -364,7 +413,6 @@ private void setHmsTableParameters(String newMetadataLocation, Table tbl, TableM
     tbl.setParameters(parameters);
   }
 
-  @VisibleForTesting

Review Comment:
   It was a mistake taken at porting the Hive: Refactor commit lock mechanism from HiveTableOperations change. I fixed it. 
   
   Thank you for finding this. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238689134


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {

Review Comment:
   Fixed.



##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236787697


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {

Review Comment:
   I think it is a porting mistake. I remove 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238709082


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238698407


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/Catalogs.java:
##########
@@ -254,47 +251,21 @@ static Optional<Catalog> loadCatalog(Configuration conf, String catalogName) {
    * @param catalogType type of the catalog
    * @return complete map of catalog properties
    */
-  private static Map<String, String> getCatalogProperties(Configuration conf, String catalogName, String catalogType) {
+  private static Map<String, String> getCatalogProperties(
+      Configuration conf, String catalogName, String catalogType) {
     Map<String, String> catalogProperties = Maps.newHashMap();
     conf.forEach(config -> {
       if (config.getKey().startsWith(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX)) {
         catalogProperties.putIfAbsent(
-            config.getKey().substring(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX.length()),
-            config.getValue());
+                config.getKey().substring(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX.length()),
+                config.getValue());
       } else if (config.getKey().startsWith(InputFormatConfig.CATALOG_CONFIG_PREFIX + catalogName)) {

Review Comment:
   Fixed.
   



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -324,19 +326,39 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           context.getProperties().get(OLD_TABLE_NAME)).toString());
     }
     if (commitLock == null) {
-      commitLock = new HiveCommitLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
-          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName());
+
+      Optional<Long> txnId = getTxnId();
+
+      commitLock = new MetastoreLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
+          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName(), txnId);
     }
 
     try {
-      commitLock.acquire();
+      commitLock.lock();
       doPreAlterTable(hmsTable, context);
     } catch (Exception e) {
-      commitLock.release();
+      commitLock.unlock();
       throw new MetaException(StringUtils.stringifyException(e));
     }
   }
 
+  @NotNull
+  private static Optional<Long> getTxnId() {

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238708545


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -473,11 +512,10 @@ private void cleanupMetadataAndUnlock(CommitStatus commitStatus, String metadata
     }
   }
 
-  @VisibleForTesting
-  void doUnlock(HiveCommitLock lock) {
+  void doUnlock(HiveLock lock) {

Review Comment:
   @deniskuzZ , can I resolve that conversation?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235256968


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -324,19 +326,39 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           context.getProperties().get(OLD_TABLE_NAME)).toString());
     }
     if (commitLock == null) {
-      commitLock = new HiveCommitLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
-          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName());
+
+      Optional<Long> txnId = getTxnId();
+
+      commitLock = new MetastoreLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),

Review Comment:
   shouldn't we create proper lock type based on hiveLockEnabled?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1239536832


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   Small query here: 
   
   > But, as 1.3.0 dependency contains those changes, with upgrading Iceberg dependency to 1.3.0, they could be removed (https://github.com/apache/hive/commit/ddba7deeb42ba8974e1cd49be96052da3ea259f4). 
   
   
   Since these files are already in iceberg 1.3, shouldn’t we remove these two lines in the pom.xml ?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1587387312

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [69 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1566865734

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] zhangbutao commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "zhangbutao (via GitHub)" <gi...@apache.org>.
zhangbutao commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1571214392

   @InvisibleProgrammer https://iceberg.apache.org/releases/#130-release Iceberg1.3.0 has been released, should we consider to upgrade to the latest version? 
   cc @deniskuzZ @ayushtkn 


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236822165


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =

Review Comment:
   As we have discussed, I reverted the 'Pass txnid to MetaStoreLock' commit



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235716497


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/TestHelpers.java:
##########
@@ -102,6 +102,24 @@ public static <T> T roundTripSerialize(T type) throws IOException, ClassNotFound
 //    );
 //  }
 
+  public static void assertSerializedMetadata(Table expected, Table actual) {
+    Assert.assertEquals("Name must match", expected.name(), actual.name());
+    Assert.assertEquals("Location must match", expected.location(), actual.location());
+    Assert.assertEquals("Props must match", expected.properties(), actual.properties());
+    Assert.assertEquals("Schema must match", expected.schema().asStruct(), actual.schema().asStruct());
+    Assert.assertEquals("Spec must match", expected.spec(), actual.spec());
+    Assert.assertEquals("Sort order must match", expected.sortOrder(), actual.sortOrder());
+  }
+
+  public static void assertSerializedAndLoadedMetadata(Table expected, Table actual) {
+    assertSerializedMetadata(expected, actual);
+    Assert.assertEquals("Specs must match", expected.specs(), actual.specs());
+    Assert.assertEquals("Sort orders must match", expected.sortOrders(), actual.sortOrders());
+    Assert.assertEquals("Current snapshot must match", expected.currentSnapshot(), actual.currentSnapshot());
+    Assert.assertEquals("Snapshots must match", expected.snapshots(), actual.snapshots());
+    Assert.assertEquals("History must match", expected.history(), actual.history());
+  }
+

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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235660530


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    metaClients.run(
+        client -> {
+          client.unlock(lockId);
+          return null;
+        });
+  }
+
+
+  private void acquireJvmLock() {
+    if (jvmLock != null) {
+      throw new IllegalStateException(String.format("Cannot call acquireLock twice for %s", fullName));
+    }
+
+    jvmLock = commitLockCache.get(fullName, t -> new ReentrantLock(true));
+    jvmLock.lock();
+  }
+
+  private void releaseJvmLock() {
+    if (jvmLock != null) {
+      jvmLock.unlock();
+      jvmLock = null;
+    }
+  }
+
+  private static void initTableLevelLockCache(long evictionTimeout) {
+    if (commitLockCache == null) {
+      synchronized (MetastoreLock.class) {
+        if (commitLockCache == null) {
+          commitLockCache =
+                  Caffeine.newBuilder()
+                          .expireAfterAccess(evictionTimeout, TimeUnit.MILLISECONDS)
+                          .build();
+        }
+      }
+    }
+  }
+
+  public String getTableName() {

Review Comment:
   No, 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235250538


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -324,19 +326,39 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           context.getProperties().get(OLD_TABLE_NAME)).toString());
     }
     if (commitLock == null) {
-      commitLock = new HiveCommitLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
-          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName());
+
+      Optional<Long> txnId = getTxnId();
+
+      commitLock = new MetastoreLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),

Review Comment:
   move txnId fetch logic in MetastoreLock constructor



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1240903811


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   `SerializableTable` in patched-iceberg-core had another method -> IncrementalAppendScan which was not present in upstream iceberg. 
   
   Can we please check and restore the SerializableTable as needed? 
   
   
   ![image](https://github.com/apache/hive/assets/26441677/89a7c068-3126-4717-8883-ee10b71ddb19)
   
   
   



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1573715327

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1589981946

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [70 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238704886


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -454,13 +502,12 @@ private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hive
     return storageDescriptor;
   }
 
-  @VisibleForTesting
-  HiveCommitLock createLock() throws UnknownHostException, TException, InterruptedException {
-    return new HiveCommitLock(conf, metaClients, catalogName, database, tableName);
+  HiveLock lockObject() {

Review Comment:
   Thank you for spotting it. 
   
   Fixed. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238699839


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   @simhadri-g , can I resolve that conversation? 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238695264


##########
iceberg/iceberg-handler/pom.xml:
##########
@@ -111,6 +111,10 @@
       <artifactId>RoaringBitmap</artifactId>
       <version>0.9.22</version>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>

Review Comment:
   Fixed.
   



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238688725


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    metaClients.run(
+        client -> {
+          client.unlock(lockId);
+          return null;
+        });
+  }
+
+
+  private void acquireJvmLock() {
+    if (jvmLock != null) {
+      throw new IllegalStateException(String.format("Cannot call acquireLock twice for %s", fullName));
+    }
+
+    jvmLock = commitLockCache.get(fullName, t -> new ReentrantLock(true));
+    jvmLock.lock();
+  }
+
+  private void releaseJvmLock() {
+    if (jvmLock != null) {
+      jvmLock.unlock();
+      jvmLock = null;
+    }
+  }
+
+  private static void initTableLevelLockCache(long evictionTimeout) {
+    if (commitLockCache == null) {
+      synchronized (MetastoreLock.class) {
+        if (commitLockCache == null) {
+          commitLockCache =
+                  Caffeine.newBuilder()
+                          .expireAfterAccess(evictionTimeout, TimeUnit.MILLISECONDS)
+                          .build();
+        }
+      }
+    }
+  }
+
+  public String getTableName() {

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238686837


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:
##########
@@ -244,11 +252,16 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
   @Override
   public void createNamespace(Namespace namespace, Map<String, String> meta) {
     Preconditions.checkArgument(
-        !namespace.isEmpty(),
-        "Cannot create namespace with invalid name: %s", namespace);
-    Preconditions.checkArgument(isValidateNamespace(namespace),
-        "Cannot support multi part namespace in Hive Metastore: %s", namespace);
-
+        !namespace.isEmpty(), "Cannot create namespace with invalid name: %s", namespace);
+    Preconditions.checkArgument(
+        isValidateNamespace(namespace),
+        "Cannot support multi part namespace in Hive Metastore: %s",

Review Comment:
   Fixed.



##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -510,6 +548,53 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
       return metadata.propertyAsBoolean(TableProperties.ENGINE_HIVE_ENABLED, false);
     }
 
-    return conf.getBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
+    return conf.getBoolean(
+        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) {
+    if (metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.HIVE_LOCK_ENABLED, false);
+    }
+
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.HIVE_LOCK_ENABLED_DEFAULT);
+  }
+
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      Optional<Long> txnId = Optional.empty();
+
+      SessionState sessionState = SessionState.get();
+      if (sessionState != null) {
+        HiveTxnManager txnMgr = sessionState.getTxnMgr();
+        if (txnMgr != null) {
+          txnId = Optional.of(txnMgr.getCurrentTxnId());
+        }
+      }
+
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName,   txnId);

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1593851978

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [84 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235204791


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/Catalogs.java:
##########
@@ -254,47 +251,21 @@ static Optional<Catalog> loadCatalog(Configuration conf, String catalogName) {
    * @param catalogType type of the catalog
    * @return complete map of catalog properties
    */
-  private static Map<String, String> getCatalogProperties(Configuration conf, String catalogName, String catalogType) {
+  private static Map<String, String> getCatalogProperties(
+      Configuration conf, String catalogName, String catalogType) {
     Map<String, String> catalogProperties = Maps.newHashMap();
     conf.forEach(config -> {
       if (config.getKey().startsWith(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX)) {
         catalogProperties.putIfAbsent(
-            config.getKey().substring(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX.length()),
-            config.getValue());
+                config.getKey().substring(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX.length()),
+                config.getValue());
       } else if (config.getKey().startsWith(InputFormatConfig.CATALOG_CONFIG_PREFIX + catalogName)) {

Review Comment:
   could we please extract `String keyPrefix = InputFormatConfig.CATALOG_CONFIG_PREFIX + catalogName;` so we won't generate new String objects on each iteration



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235249683


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -324,19 +326,39 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           context.getProperties().get(OLD_TABLE_NAME)).toString());
     }
     if (commitLock == null) {
-      commitLock = new HiveCommitLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
-          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName());
+
+      Optional<Long> txnId = getTxnId();
+
+      commitLock = new MetastoreLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
+          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName(), txnId);
     }
 
     try {
-      commitLock.acquire();
+      commitLock.lock();
       doPreAlterTable(hmsTable, context);
     } catch (Exception e) {
-      commitLock.release();
+      commitLock.unlock();
       throw new MetaException(StringUtils.stringifyException(e));
     }
   }
 
+  @NotNull
+  private static Optional<Long> getTxnId() {

Review Comment:
   same code was added multiple times



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ayushtkn commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "ayushtkn (via GitHub)" <gi...@apache.org>.
ayushtkn commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235156198


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -337,6 +342,23 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
     }
   }
 
+  @NotNull
+  private static Optional<Long> getTxnId() {
+    Optional<Long> txnId;
+    txnId = Optional.empty();
+
+    SessionState sessionState = SessionState.get();
+
+    if (sessionState != null) {
+      HiveTxnManager txnMgr = sessionState.getTxnMgr();
+      if (txnMgr != null) {
+        txnId = Optional.of(txnMgr.getCurrentTxnId());
+      }
+    }

Review Comment:
   this part is dupe in
   ```
   HiveTableOperations.java
   ```
   and this in static method, can we keep this logic at one place & use it in the another?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236639408


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   @zsmiskolczi  please correct me if i am wrong.
   
   Patched-iceberg-core package need to have the patched TableProperties.class and ConfigProperties.class files mentioned here in it.
   Currently in this PR, we are missing the patched files... Is there a different PR for these patched files?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1604096028

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235187827


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -364,7 +413,6 @@ private void setHmsTableParameters(String newMetadataLocation, Table tbl, TableM
     tbl.setParameters(parameters);
   }
 
-  @VisibleForTesting

Review Comment:
   Why is this annotation removed? I see the same in upstream Iceberg 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235873970


##########
iceberg/iceberg-handler/pom.xml:
##########
@@ -111,6 +111,10 @@
       <artifactId>RoaringBitmap</artifactId>
       <version>0.9.22</version>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>

Review Comment:
   It was in the list of commits for 1.3.0. But yes, it is unused in iceberg handler. 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1237075772


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -454,13 +502,12 @@ private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hive
     return storageDescriptor;
   }
 
-  @VisibleForTesting
-  HiveCommitLock createLock() throws UnknownHostException, TException, InterruptedException {
-    return new HiveCommitLock(conf, metaClients, catalogName, database, tableName);
+  HiveLock lockObject() {

Review Comment:
   wasn't it replaced with `HiveLock lockObject(TableMetadata metadata)`



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235268025


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -473,11 +512,10 @@ private void cleanupMetadataAndUnlock(CommitStatus commitStatus, String metadata
     }
   }
 
-  @VisibleForTesting
-  void doUnlock(HiveCommitLock lock) {
+  void doUnlock(HiveLock lock) {

Review Comment:
   ok, what is the use of doUnlock() if we just swallow the exception?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235315052


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java:
##########
@@ -21,25 +21,61 @@
 
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
 import org.apache.thrift.TException;
+import org.immutables.value.Value;
 
+/**
+ * A ClientPool that caches the underlying HiveClientPool instances.
+ *
+ * <p>The following key elements are supported and can be specified via {@link
+ * CatalogProperties#CLIENT_POOL_CACHE_KEYS}:
+ *
+ * <ul>
+ *   <li>ugi - the Hadoop UserGroupInformation instance that represents the current user using the
+ *       cache.
+ *   <li>user_name - similar to UGI but only includes the user's name determined by
+ *       UserGroupInformation#getUserName.
+ *   <li>conf - name of an arbitrary configuration. The value of the configuration will be extracted
+ *       from catalog properties and added to the cache key. A conf element should start with a
+ *       "conf:" prefix which is followed by the configuration name. E.g. specifying "conf:a.b.c"
+ *       will add "a.b.c" to the key, and so that configurations with different default catalog
+ *       wouldn't share the same client pool. Multiple conf elements can be specified.
+ * </ul>
+ */
 public class CachedClientPool implements ClientPool<IMetaStoreClient, TException> {
 
-  private static Cache<String, HiveClientPool> clientPoolCache;
+  private static final String CONF_ELEMENT_PREFIX = "conf:";
+
+  private static Cache<Key, HiveClientPool> clientPoolCache;
 
   private final Configuration conf;
   private final String metastoreUri;

Review Comment:
   Ohh, got it. Removed in https://github.com/apache/hive/pull/4291/commits/8e024ff25019a8c04b69aeb1a82a5b58e31e99bc
   
   Thank you.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235018441


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -473,11 +512,10 @@ private void cleanupMetadataAndUnlock(CommitStatus commitStatus, String metadata
     }
   }
 
-  @VisibleForTesting
-  void doUnlock(HiveCommitLock lock) {
+  void doUnlock(HiveLock lock) {

Review Comment:
   is that used? Looks like it was replaced with `cleanupMetadataAndUnlock` in `Hive: Refactor commit lock mechanism from HiveTableOperations (#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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235095817


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.

Review Comment:
   is this comment still valid?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1607785590

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1585055031

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [69 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1557520706

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1592014509

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [71 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235901885


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java:
##########
@@ -205,6 +205,11 @@ public void testResiduals() throws Exception {
 
   @Test
   public void testFailedResidualFiltering() throws Exception {
+    // Vectorization is not yet supported for AVRO
+    if (this.fileFormat.equals(FileFormat.AVRO)) {

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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1240904457


##########
iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/SerializableTable.java:
##########
@@ -1,420 +0,0 @@
-/*
- * 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;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.iceberg.encryption.EncryptionManager;
-import org.apache.iceberg.hadoop.HadoopConfigurable;
-import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.io.LocationProvider;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.SerializableMap;
-import org.apache.iceberg.util.SerializableSupplier;
-
-/**
- * A read-only serializable table that can be sent to other nodes in a cluster.
- *
- * <p>An instance of this class represents an immutable serializable copy of a table state and will
- * not reflect any subsequent changed made to the original table.
- *
- * <p>While this class captures the metadata file location that can be used to load the complete
- * table metadata, it directly persists the current schema, spec, sort order, table properties to
- * avoid reading the metadata file from other nodes for frequently needed metadata.
- *
- * <p>The implementation assumes the passed instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} are serializable. If you are serializing the table using a custom
- * serialization framework like Kryo, those instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} must be supported by that particular serialization framework.
- *
- * <p><em>Note:</em> loading the complete metadata from a large number of nodes can overwhelm the
- * storage.
- */
-public class SerializableTable implements Table, Serializable {
-
-  private final String name;
-  private final String location;
-  private final String metadataFileLocation;
-  private final Map<String, String> properties;
-  private final String schemaAsJson;
-  private final int defaultSpecId;
-  private final Map<Integer, String> specAsJsonMap;
-  private final String sortOrderAsJson;
-  private final FileIO io;
-  private final EncryptionManager encryption;
-  private final LocationProvider locationProvider;
-  private final Map<String, SnapshotRef> refs;
-
-  private transient volatile Table lazyTable = null;
-  private transient volatile Schema lazySchema = null;
-  private transient volatile Map<Integer, PartitionSpec> lazySpecs = null;
-  private transient volatile SortOrder lazySortOrder = null;
-
-  protected SerializableTable(Table table) {
-    this.name = table.name();
-    this.location = table.location();
-    this.metadataFileLocation = metadataFileLocation(table);
-    this.properties = SerializableMap.copyOf(table.properties());
-    this.schemaAsJson = SchemaParser.toJson(table.schema());
-    this.defaultSpecId = table.spec().specId();
-    this.specAsJsonMap = Maps.newHashMap();
-    Map<Integer, PartitionSpec> specs = table.specs();
-    specs.forEach((specId, spec) -> specAsJsonMap.put(specId, PartitionSpecParser.toJson(spec)));
-    this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder());
-    this.io = fileIO(table);
-    this.encryption = table.encryption();
-    this.locationProvider = table.locationProvider();
-    this.refs = SerializableMap.copyOf(table.refs());
-  }
-
-  /**
-   * Creates a read-only serializable table that can be sent to other nodes in a cluster.
-   *
-   * @param table the original table to copy the state from
-   * @return a read-only serializable table reflecting the current state of the original table
-   */
-  public static Table copyOf(Table table) {
-    if (table instanceof BaseMetadataTable) {
-      return new SerializableMetadataTable((BaseMetadataTable) table);
-    } else {
-      return new SerializableTable(table);
-    }
-  }
-
-  private String metadataFileLocation(Table table) {
-    if (table instanceof HasTableOperations) {
-      TableOperations ops = ((HasTableOperations) table).operations();
-      return ops.current().metadataFileLocation();
-    } else {
-      return null;
-    }
-  }
-
-  private FileIO fileIO(Table table) {
-    if (table.io() instanceof HadoopConfigurable) {
-      ((HadoopConfigurable) table.io()).serializeConfWith(SerializableConfSupplier::new);
-    }
-
-    return table.io();
-  }
-
-  private Table lazyTable() {
-    if (lazyTable == null) {
-      synchronized (this) {
-        if (lazyTable == null) {
-          if (metadataFileLocation == null) {
-            throw new UnsupportedOperationException(
-                    "Cannot load metadata: metadata file location is null");
-          }
-
-          TableOperations ops =
-                  new StaticTableOperations(metadataFileLocation, io, locationProvider);
-          this.lazyTable = newTable(ops, name);
-        }
-      }
-    }
-
-    return lazyTable;
-  }
-
-  protected Table newTable(TableOperations ops, String tableName) {
-    return new BaseTable(ops, tableName);
-  }
-
-  @Override
-  public String name() {
-    return name;
-  }
-
-  @Override
-  public String location() {
-    return location;
-  }
-
-  @Override
-  public Map<String, String> properties() {
-    return properties;
-  }
-
-  @Override
-  public Schema schema() {
-    if (lazySchema == null) {
-      synchronized (this) {
-        if (lazySchema == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySchema = SchemaParser.fromJson(schemaAsJson);
-        } else if (lazySchema == null) {
-          this.lazySchema = lazyTable.schema();
-        }
-      }
-    }
-
-    return lazySchema;
-  }
-
-  @Override
-  public Map<Integer, Schema> schemas() {
-    return lazyTable().schemas();
-  }
-
-  @Override
-  public PartitionSpec spec() {
-    return specs().get(defaultSpecId);
-  }
-
-  @Override
-  public Map<Integer, PartitionSpec> specs() {
-    if (lazySpecs == null) {
-      synchronized (this) {
-        if (lazySpecs == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          Map<Integer, PartitionSpec> specs = Maps.newHashMapWithExpectedSize(specAsJsonMap.size());
-          specAsJsonMap.forEach(
-              (specId, specAsJson) -> {
-                specs.put(specId, PartitionSpecParser.fromJson(schema(), specAsJson));
-              });
-          this.lazySpecs = specs;
-        } else if (lazySpecs == null) {
-          this.lazySpecs = lazyTable.specs();
-        }
-      }
-    }
-
-    return lazySpecs;
-  }
-
-  @Override
-  public SortOrder sortOrder() {
-    if (lazySortOrder == null) {
-      synchronized (this) {
-        if (lazySortOrder == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySortOrder = SortOrderParser.fromJson(schema(), sortOrderAsJson);
-        } else if (lazySortOrder == null) {
-          this.lazySortOrder = lazyTable.sortOrder();
-        }
-      }
-    }
-
-    return lazySortOrder;
-  }
-
-  @Override
-  public Map<Integer, SortOrder> sortOrders() {
-    return lazyTable().sortOrders();
-  }
-
-  @Override
-  public FileIO io() {
-    return io;
-  }
-
-  @Override
-  public EncryptionManager encryption() {
-    return encryption;
-  }
-
-  @Override
-  public LocationProvider locationProvider() {
-    return locationProvider;
-  }
-
-  @Override
-  public List<StatisticsFile> statisticsFiles() {
-    return lazyTable().statisticsFiles();
-  }
-
-  @Override
-  public Map<String, SnapshotRef> refs() {
-    return refs;
-  }
-
-  @Override
-  public void refresh() {
-    throw new UnsupportedOperationException(errorMsg("refresh"));
-  }
-
-  @Override
-  public TableScan newScan() {
-    return lazyTable().newScan();
-  }
-
-  public IncrementalAppendScan newIncrementalAppendScan() {

Review Comment:
   Can we reconfirm this and restore SerializableTable if needed?
   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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1240903811


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   `SerializableTable` in patched-iceberg-core had another method -> IncrementalAppendScan which was not present in upstream iceberg. 
   
   
   Comparing the diff between the patched-iceberg-core-SerializableTable in hive and org.apache.iceberg.SerializableTable in iceberg .
   ![image](https://github.com/apache/hive/assets/26441677/89a7c068-3126-4717-8883-ee10b71ddb19)
   
   Can we please check and restore the SerializableTable as needed? 
   



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238698786


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -324,19 +326,39 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           context.getProperties().get(OLD_TABLE_NAME)).toString());
     }
     if (commitLock == null) {
-      commitLock = new HiveCommitLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
-          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName());
+
+      Optional<Long> txnId = getTxnId();
+
+      commitLock = new MetastoreLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238696363


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -364,7 +413,6 @@ private void setHmsTableParameters(String newMetadataLocation, Table tbl, TableM
     tbl.setParameters(parameters);
   }
 
-  @VisibleForTesting

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ayushtkn commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "ayushtkn (via GitHub)" <gi...@apache.org>.
ayushtkn commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235174915


##########
iceberg/iceberg-handler/pom.xml:
##########
@@ -111,6 +111,10 @@
       <artifactId>RoaringBitmap</artifactId>
       <version>0.9.22</version>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>

Review Comment:
   Why is this required? where is iceberg handler using this? 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236787697


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {

Review Comment:
   I think it is a porting mistake. I removed 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1241615094


##########
iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/SerializableTable.java:
##########
@@ -1,420 +0,0 @@
-/*
- * 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;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.iceberg.encryption.EncryptionManager;
-import org.apache.iceberg.hadoop.HadoopConfigurable;
-import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.io.LocationProvider;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.SerializableMap;
-import org.apache.iceberg.util.SerializableSupplier;
-
-/**
- * A read-only serializable table that can be sent to other nodes in a cluster.
- *
- * <p>An instance of this class represents an immutable serializable copy of a table state and will
- * not reflect any subsequent changed made to the original table.
- *
- * <p>While this class captures the metadata file location that can be used to load the complete
- * table metadata, it directly persists the current schema, spec, sort order, table properties to
- * avoid reading the metadata file from other nodes for frequently needed metadata.
- *
- * <p>The implementation assumes the passed instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} are serializable. If you are serializing the table using a custom
- * serialization framework like Kryo, those instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} must be supported by that particular serialization framework.
- *
- * <p><em>Note:</em> loading the complete metadata from a large number of nodes can overwhelm the
- * storage.
- */
-public class SerializableTable implements Table, Serializable {
-
-  private final String name;
-  private final String location;
-  private final String metadataFileLocation;
-  private final Map<String, String> properties;
-  private final String schemaAsJson;
-  private final int defaultSpecId;
-  private final Map<Integer, String> specAsJsonMap;
-  private final String sortOrderAsJson;
-  private final FileIO io;
-  private final EncryptionManager encryption;
-  private final LocationProvider locationProvider;
-  private final Map<String, SnapshotRef> refs;
-
-  private transient volatile Table lazyTable = null;
-  private transient volatile Schema lazySchema = null;
-  private transient volatile Map<Integer, PartitionSpec> lazySpecs = null;
-  private transient volatile SortOrder lazySortOrder = null;
-
-  protected SerializableTable(Table table) {
-    this.name = table.name();
-    this.location = table.location();
-    this.metadataFileLocation = metadataFileLocation(table);
-    this.properties = SerializableMap.copyOf(table.properties());
-    this.schemaAsJson = SchemaParser.toJson(table.schema());
-    this.defaultSpecId = table.spec().specId();
-    this.specAsJsonMap = Maps.newHashMap();
-    Map<Integer, PartitionSpec> specs = table.specs();
-    specs.forEach((specId, spec) -> specAsJsonMap.put(specId, PartitionSpecParser.toJson(spec)));
-    this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder());
-    this.io = fileIO(table);
-    this.encryption = table.encryption();
-    this.locationProvider = table.locationProvider();
-    this.refs = SerializableMap.copyOf(table.refs());
-  }
-
-  /**
-   * Creates a read-only serializable table that can be sent to other nodes in a cluster.
-   *
-   * @param table the original table to copy the state from
-   * @return a read-only serializable table reflecting the current state of the original table
-   */
-  public static Table copyOf(Table table) {
-    if (table instanceof BaseMetadataTable) {
-      return new SerializableMetadataTable((BaseMetadataTable) table);
-    } else {
-      return new SerializableTable(table);
-    }
-  }
-
-  private String metadataFileLocation(Table table) {
-    if (table instanceof HasTableOperations) {
-      TableOperations ops = ((HasTableOperations) table).operations();
-      return ops.current().metadataFileLocation();
-    } else {
-      return null;
-    }
-  }
-
-  private FileIO fileIO(Table table) {
-    if (table.io() instanceof HadoopConfigurable) {
-      ((HadoopConfigurable) table.io()).serializeConfWith(SerializableConfSupplier::new);
-    }
-
-    return table.io();
-  }
-
-  private Table lazyTable() {
-    if (lazyTable == null) {
-      synchronized (this) {
-        if (lazyTable == null) {
-          if (metadataFileLocation == null) {
-            throw new UnsupportedOperationException(
-                    "Cannot load metadata: metadata file location is null");
-          }
-
-          TableOperations ops =
-                  new StaticTableOperations(metadataFileLocation, io, locationProvider);
-          this.lazyTable = newTable(ops, name);
-        }
-      }
-    }
-
-    return lazyTable;
-  }
-
-  protected Table newTable(TableOperations ops, String tableName) {
-    return new BaseTable(ops, tableName);
-  }
-
-  @Override
-  public String name() {
-    return name;
-  }
-
-  @Override
-  public String location() {
-    return location;
-  }
-
-  @Override
-  public Map<String, String> properties() {
-    return properties;
-  }
-
-  @Override
-  public Schema schema() {
-    if (lazySchema == null) {
-      synchronized (this) {
-        if (lazySchema == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySchema = SchemaParser.fromJson(schemaAsJson);
-        } else if (lazySchema == null) {
-          this.lazySchema = lazyTable.schema();
-        }
-      }
-    }
-
-    return lazySchema;
-  }
-
-  @Override
-  public Map<Integer, Schema> schemas() {
-    return lazyTable().schemas();
-  }
-
-  @Override
-  public PartitionSpec spec() {
-    return specs().get(defaultSpecId);
-  }
-
-  @Override
-  public Map<Integer, PartitionSpec> specs() {
-    if (lazySpecs == null) {
-      synchronized (this) {
-        if (lazySpecs == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          Map<Integer, PartitionSpec> specs = Maps.newHashMapWithExpectedSize(specAsJsonMap.size());
-          specAsJsonMap.forEach(
-              (specId, specAsJson) -> {
-                specs.put(specId, PartitionSpecParser.fromJson(schema(), specAsJson));
-              });
-          this.lazySpecs = specs;
-        } else if (lazySpecs == null) {
-          this.lazySpecs = lazyTable.specs();
-        }
-      }
-    }
-
-    return lazySpecs;
-  }
-
-  @Override
-  public SortOrder sortOrder() {
-    if (lazySortOrder == null) {
-      synchronized (this) {
-        if (lazySortOrder == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySortOrder = SortOrderParser.fromJson(schema(), sortOrderAsJson);
-        } else if (lazySortOrder == null) {
-          this.lazySortOrder = lazyTable.sortOrder();
-        }
-      }
-    }
-
-    return lazySortOrder;
-  }
-
-  @Override
-  public Map<Integer, SortOrder> sortOrders() {
-    return lazyTable().sortOrders();
-  }
-
-  @Override
-  public FileIO io() {
-    return io;
-  }
-
-  @Override
-  public EncryptionManager encryption() {
-    return encryption;
-  }
-
-  @Override
-  public LocationProvider locationProvider() {
-    return locationProvider;
-  }
-
-  @Override
-  public List<StatisticsFile> statisticsFiles() {
-    return lazyTable().statisticsFiles();
-  }
-
-  @Override
-  public Map<String, SnapshotRef> refs() {
-    return refs;
-  }
-
-  @Override
-  public void refresh() {
-    throw new UnsupportedOperationException(errorMsg("refresh"));
-  }
-
-  @Override
-  public TableScan newScan() {
-    return lazyTable().newScan();
-  }
-
-  public IncrementalAppendScan newIncrementalAppendScan() {

Review Comment:
   it's used in IcebergInputFormat
   ````
     private static IncrementalAppendScan createIncrementalAppendScan(Table table, Configuration conf) {
       long fromSnapshot = conf.getLong(InputFormatConfig.SNAPSHOT_ID_INTERVAL_FROM, -1);
       return table.newIncrementalAppendScan().fromSnapshotExclusive(fromSnapshot);
     }
   ````
   we should have it in iceberg/patched-iceberg-core/pom.xml



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1608357449

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1553650890

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "szlta (via GitHub)" <gi...@apache.org>.
szlta commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1541551847

   > > First of all, I have a question about the namings: do you know why Iceberg and Hive has different naming for the class using for locking? In Hive, it is called `HiveCommitLog` and in Iceberg, it is `HiveMetastoreLock`.
   > 
   > It was after my time, but I expect that @szlta give up on getting his refactor into Iceberg, but needed it in Hive.
   
   Yup, I did that refactor with https://github.com/apache/hive/commit/2e916a3ec26a6d12653c89ad445fed91e1ba0cde so that we can acquire the commit lock from `HiveIcebergMetaHook` too. The reason why it was not ported to Iceberg was that Iceberg was depending on an old Hive version where some of the hooks (especially for alter_table) didn't exist yet.


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235144768


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -510,6 +548,53 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
       return metadata.propertyAsBoolean(TableProperties.ENGINE_HIVE_ENABLED, false);
     }
 
-    return conf.getBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
+    return conf.getBoolean(
+        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) {
+    if (metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.HIVE_LOCK_ENABLED, false);
+    }
+
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.HIVE_LOCK_ENABLED_DEFAULT);
+  }
+
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      Optional<Long> txnId = Optional.empty();
+
+      SessionState sessionState = SessionState.get();
+      if (sessionState != null) {
+        HiveTxnManager txnMgr = sessionState.getTxnMgr();
+        if (txnMgr != null) {
+          txnId = Optional.of(txnMgr.getCurrentTxnId());
+        }
+      }
+
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName,   txnId);

Review Comment:
   For that one, unfortunately I cannot: This method is only used by tests and those tests doesn't use MetastoreLock. 
   For all the other use cases, it gets txnId in HiveIcebergMetaHook when it creates the MetastoreLock instance. 
   
   <img width="877" alt="image" src="https://github.com/apache/hive/assets/1486749/0341c259-ebc3-4756-889d-ad944f36fe1e">
   



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235147278


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -473,11 +512,10 @@ private void cleanupMetadataAndUnlock(CommitStatus commitStatus, String metadata
     }
   }
 
-  @VisibleForTesting
-  void doUnlock(HiveCommitLock lock) {
+  void doUnlock(HiveLock lock) {

Review Comment:
   Yes, it is used in `cleanupMetadataAndUnlock`. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1603270859

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238686485


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java:
##########
@@ -21,25 +21,61 @@
 
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
 import org.apache.thrift.TException;
+import org.immutables.value.Value;
 
+/**
+ * A ClientPool that caches the underlying HiveClientPool instances.
+ *
+ * <p>The following key elements are supported and can be specified via {@link
+ * CatalogProperties#CLIENT_POOL_CACHE_KEYS}:
+ *
+ * <ul>
+ *   <li>ugi - the Hadoop UserGroupInformation instance that represents the current user using the
+ *       cache.
+ *   <li>user_name - similar to UGI but only includes the user's name determined by
+ *       UserGroupInformation#getUserName.
+ *   <li>conf - name of an arbitrary configuration. The value of the configuration will be extracted
+ *       from catalog properties and added to the cache key. A conf element should start with a
+ *       "conf:" prefix which is followed by the configuration name. E.g. specifying "conf:a.b.c"
+ *       will add "a.b.c" to the key, and so that configurations with different default catalog
+ *       wouldn't share the same client pool. Multiple conf elements can be specified.
+ * </ul>
+ */
 public class CachedClientPool implements ClientPool<IMetaStoreClient, TException> {
 
-  private static Cache<String, HiveClientPool> clientPoolCache;
+  private static final String CONF_ELEMENT_PREFIX = "conf:";
+
+  private static Cache<Key, HiveClientPool> clientPoolCache;
 
   private final Configuration conf;
   private final String metastoreUri;

Review Comment:
   Fixed



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238690483


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/TestHelpers.java:
##########
@@ -102,6 +102,24 @@ public static <T> T roundTripSerialize(T type) throws IOException, ClassNotFound
 //    );
 //  }
 
+  public static void assertSerializedMetadata(Table expected, Table actual) {
+    Assert.assertEquals("Name must match", expected.name(), actual.name());
+    Assert.assertEquals("Location must match", expected.location(), actual.location());
+    Assert.assertEquals("Props must match", expected.properties(), actual.properties());
+    Assert.assertEquals("Schema must match", expected.schema().asStruct(), actual.schema().asStruct());
+    Assert.assertEquals("Spec must match", expected.spec(), actual.spec());
+    Assert.assertEquals("Sort order must match", expected.sortOrder(), actual.sortOrder());
+  }
+
+  public static void assertSerializedAndLoadedMetadata(Table expected, Table actual) {
+    assertSerializedMetadata(expected, actual);
+    Assert.assertEquals("Specs must match", expected.specs(), actual.specs());
+    Assert.assertEquals("Sort orders must match", expected.sortOrders(), actual.sortOrders());
+    Assert.assertEquals("Current snapshot must match", expected.currentSnapshot(), actual.currentSnapshot());
+    Assert.assertEquals("Snapshots must match", expected.snapshots(), actual.snapshots());
+    Assert.assertEquals("History must match", expected.history(), actual.history());
+  }
+

Review Comment:
   I close it. @ayushtkn , feel free to reopen it if it is still have to be discussed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238691237


##########
iceberg/iceberg-catalog/pom.xml:
##########
@@ -74,5 +78,10 @@
       <artifactId>junit-vintage-engine</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>

Review Comment:
   Fixed.
   



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -337,6 +342,23 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
     }
   }
 
+  @NotNull
+  private static Optional<Long> getTxnId() {
+    Optional<Long> txnId;
+    txnId = Optional.empty();
+
+    SessionState sessionState = SessionState.get();
+
+    if (sessionState != null) {
+      HiveTxnManager txnMgr = sessionState.getTxnMgr();
+      if (txnMgr != null) {
+        txnId = Optional.of(txnMgr.getCurrentTxnId());
+      }
+    }

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235154512


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -176,11 +180,10 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
     CommitStatus commitStatus = CommitStatus.FAILURE;
     boolean updateHiveTable = false;
-    HiveCommitLock commitLock = null;
 
+    HiveLock lock = lockObject(metadata);

Review Comment:
   Nice catch! I don't know how, but I have skipped it. Let me add to the list. It was part of the 1.2.1 release. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236784319


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;

Review Comment:
   As we have discussed, I reverted the 'Pass txnid to MetaStoreLock' commit



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ayushtkn commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "ayushtkn (via GitHub)" <gi...@apache.org>.
ayushtkn commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235136608


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/TestHelpers.java:
##########
@@ -102,6 +102,24 @@ public static <T> T roundTripSerialize(T type) throws IOException, ClassNotFound
 //    );
 //  }
 
+  public static void assertSerializedMetadata(Table expected, Table actual) {
+    Assert.assertEquals("Name must match", expected.name(), actual.name());
+    Assert.assertEquals("Location must match", expected.location(), actual.location());
+    Assert.assertEquals("Props must match", expected.properties(), actual.properties());
+    Assert.assertEquals("Schema must match", expected.schema().asStruct(), actual.schema().asStruct());
+    Assert.assertEquals("Spec must match", expected.spec(), actual.spec());
+    Assert.assertEquals("Sort order must match", expected.sortOrder(), actual.sortOrder());
+  }
+
+  public static void assertSerializedAndLoadedMetadata(Table expected, Table actual) {
+    assertSerializedMetadata(expected, actual);
+    Assert.assertEquals("Specs must match", expected.specs(), actual.specs());
+    Assert.assertEquals("Sort orders must match", expected.sortOrders(), actual.sortOrders());
+    Assert.assertEquals("Current snapshot must match", expected.currentSnapshot(), actual.currentSnapshot());
+    Assert.assertEquals("Snapshots must match", expected.snapshots(), actual.snapshots());
+    Assert.assertEquals("History must match", expected.history(), actual.history());
+  }
+

Review Comment:
   this entire commit seems of no use, the usage doesn't lies in our code, we can drop this?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235716497


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/TestHelpers.java:
##########
@@ -102,6 +102,24 @@ public static <T> T roundTripSerialize(T type) throws IOException, ClassNotFound
 //    );
 //  }
 
+  public static void assertSerializedMetadata(Table expected, Table actual) {
+    Assert.assertEquals("Name must match", expected.name(), actual.name());
+    Assert.assertEquals("Location must match", expected.location(), actual.location());
+    Assert.assertEquals("Props must match", expected.properties(), actual.properties());
+    Assert.assertEquals("Schema must match", expected.schema().asStruct(), actual.schema().asStruct());
+    Assert.assertEquals("Spec must match", expected.spec(), actual.spec());
+    Assert.assertEquals("Sort order must match", expected.sortOrder(), actual.sortOrder());
+  }
+
+  public static void assertSerializedAndLoadedMetadata(Table expected, Table actual) {
+    assertSerializedMetadata(expected, actual);
+    Assert.assertEquals("Specs must match", expected.specs(), actual.specs());
+    Assert.assertEquals("Sort orders must match", expected.sortOrders(), actual.sortOrders());
+    Assert.assertEquals("Current snapshot must match", expected.currentSnapshot(), actual.currentSnapshot());
+    Assert.assertEquals("Snapshots must match", expected.snapshots(), actual.snapshots());
+    Assert.assertEquals("History must match", expected.history(), actual.history());
+  }
+

Review Comment:
   It is called from `TestHiveCatalog.testRegisterTable.` Fails compilation without 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1240903811


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   `SerializableTable` in patched-iceberg-core had another method -> IncrementalAppendScan which was not present in upstream iceberg. 
   Can we please check and restore the SerializableTable as needed? 
   



##########
iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/SerializableTable.java:
##########
@@ -1,420 +0,0 @@
-/*
- * 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;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.iceberg.encryption.EncryptionManager;
-import org.apache.iceberg.hadoop.HadoopConfigurable;
-import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.io.LocationProvider;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.SerializableMap;
-import org.apache.iceberg.util.SerializableSupplier;
-
-/**
- * A read-only serializable table that can be sent to other nodes in a cluster.
- *
- * <p>An instance of this class represents an immutable serializable copy of a table state and will
- * not reflect any subsequent changed made to the original table.
- *
- * <p>While this class captures the metadata file location that can be used to load the complete
- * table metadata, it directly persists the current schema, spec, sort order, table properties to
- * avoid reading the metadata file from other nodes for frequently needed metadata.
- *
- * <p>The implementation assumes the passed instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} are serializable. If you are serializing the table using a custom
- * serialization framework like Kryo, those instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} must be supported by that particular serialization framework.
- *
- * <p><em>Note:</em> loading the complete metadata from a large number of nodes can overwhelm the
- * storage.
- */
-public class SerializableTable implements Table, Serializable {
-
-  private final String name;
-  private final String location;
-  private final String metadataFileLocation;
-  private final Map<String, String> properties;
-  private final String schemaAsJson;
-  private final int defaultSpecId;
-  private final Map<Integer, String> specAsJsonMap;
-  private final String sortOrderAsJson;
-  private final FileIO io;
-  private final EncryptionManager encryption;
-  private final LocationProvider locationProvider;
-  private final Map<String, SnapshotRef> refs;
-
-  private transient volatile Table lazyTable = null;
-  private transient volatile Schema lazySchema = null;
-  private transient volatile Map<Integer, PartitionSpec> lazySpecs = null;
-  private transient volatile SortOrder lazySortOrder = null;
-
-  protected SerializableTable(Table table) {
-    this.name = table.name();
-    this.location = table.location();
-    this.metadataFileLocation = metadataFileLocation(table);
-    this.properties = SerializableMap.copyOf(table.properties());
-    this.schemaAsJson = SchemaParser.toJson(table.schema());
-    this.defaultSpecId = table.spec().specId();
-    this.specAsJsonMap = Maps.newHashMap();
-    Map<Integer, PartitionSpec> specs = table.specs();
-    specs.forEach((specId, spec) -> specAsJsonMap.put(specId, PartitionSpecParser.toJson(spec)));
-    this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder());
-    this.io = fileIO(table);
-    this.encryption = table.encryption();
-    this.locationProvider = table.locationProvider();
-    this.refs = SerializableMap.copyOf(table.refs());
-  }
-
-  /**
-   * Creates a read-only serializable table that can be sent to other nodes in a cluster.
-   *
-   * @param table the original table to copy the state from
-   * @return a read-only serializable table reflecting the current state of the original table
-   */
-  public static Table copyOf(Table table) {
-    if (table instanceof BaseMetadataTable) {
-      return new SerializableMetadataTable((BaseMetadataTable) table);
-    } else {
-      return new SerializableTable(table);
-    }
-  }
-
-  private String metadataFileLocation(Table table) {
-    if (table instanceof HasTableOperations) {
-      TableOperations ops = ((HasTableOperations) table).operations();
-      return ops.current().metadataFileLocation();
-    } else {
-      return null;
-    }
-  }
-
-  private FileIO fileIO(Table table) {
-    if (table.io() instanceof HadoopConfigurable) {
-      ((HadoopConfigurable) table.io()).serializeConfWith(SerializableConfSupplier::new);
-    }
-
-    return table.io();
-  }
-
-  private Table lazyTable() {
-    if (lazyTable == null) {
-      synchronized (this) {
-        if (lazyTable == null) {
-          if (metadataFileLocation == null) {
-            throw new UnsupportedOperationException(
-                    "Cannot load metadata: metadata file location is null");
-          }
-
-          TableOperations ops =
-                  new StaticTableOperations(metadataFileLocation, io, locationProvider);
-          this.lazyTable = newTable(ops, name);
-        }
-      }
-    }
-
-    return lazyTable;
-  }
-
-  protected Table newTable(TableOperations ops, String tableName) {
-    return new BaseTable(ops, tableName);
-  }
-
-  @Override
-  public String name() {
-    return name;
-  }
-
-  @Override
-  public String location() {
-    return location;
-  }
-
-  @Override
-  public Map<String, String> properties() {
-    return properties;
-  }
-
-  @Override
-  public Schema schema() {
-    if (lazySchema == null) {
-      synchronized (this) {
-        if (lazySchema == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySchema = SchemaParser.fromJson(schemaAsJson);
-        } else if (lazySchema == null) {
-          this.lazySchema = lazyTable.schema();
-        }
-      }
-    }
-
-    return lazySchema;
-  }
-
-  @Override
-  public Map<Integer, Schema> schemas() {
-    return lazyTable().schemas();
-  }
-
-  @Override
-  public PartitionSpec spec() {
-    return specs().get(defaultSpecId);
-  }
-
-  @Override
-  public Map<Integer, PartitionSpec> specs() {
-    if (lazySpecs == null) {
-      synchronized (this) {
-        if (lazySpecs == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          Map<Integer, PartitionSpec> specs = Maps.newHashMapWithExpectedSize(specAsJsonMap.size());
-          specAsJsonMap.forEach(
-              (specId, specAsJson) -> {
-                specs.put(specId, PartitionSpecParser.fromJson(schema(), specAsJson));
-              });
-          this.lazySpecs = specs;
-        } else if (lazySpecs == null) {
-          this.lazySpecs = lazyTable.specs();
-        }
-      }
-    }
-
-    return lazySpecs;
-  }
-
-  @Override
-  public SortOrder sortOrder() {
-    if (lazySortOrder == null) {
-      synchronized (this) {
-        if (lazySortOrder == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySortOrder = SortOrderParser.fromJson(schema(), sortOrderAsJson);
-        } else if (lazySortOrder == null) {
-          this.lazySortOrder = lazyTable.sortOrder();
-        }
-      }
-    }
-
-    return lazySortOrder;
-  }
-
-  @Override
-  public Map<Integer, SortOrder> sortOrders() {
-    return lazyTable().sortOrders();
-  }
-
-  @Override
-  public FileIO io() {
-    return io;
-  }
-
-  @Override
-  public EncryptionManager encryption() {
-    return encryption;
-  }
-
-  @Override
-  public LocationProvider locationProvider() {
-    return locationProvider;
-  }
-
-  @Override
-  public List<StatisticsFile> statisticsFiles() {
-    return lazyTable().statisticsFiles();
-  }
-
-  @Override
-  public Map<String, SnapshotRef> refs() {
-    return refs;
-  }
-
-  @Override
-  public void refresh() {
-    throw new UnsupportedOperationException(errorMsg("refresh"));
-  }
-
-  @Override
-  public TableScan newScan() {
-    return lazyTable().newScan();
-  }
-
-  public IncrementalAppendScan newIncrementalAppendScan() {

Review Comment:
   Comparing the diff between the patched-iceberg-core-SerializableTable in hive and org.apache.iceberg.SerializableTable in iceberg .
   ![image](https://github.com/apache/hive/assets/26441677/89a7c068-3126-4717-8883-ee10b71ddb19)
   



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1572784779

   > @InvisibleProgrammer https://iceberg.apache.org/releases/#130-release Iceberg1.3.0 has been released, should we consider to upgrade to the latest version? cc @deniskuzZ @ayushtkn
   
   I bet we should upgrade. But I'm not sure if we should do it in this PR. I think, we have to finish 1.2.1 first. When it is merged, upgrade the dependencies to 1.3.0. And after that, port the catalog and handler (mr) changes.
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1551121502

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1537887734

   Hi, @pvary . 
   
   This is the PR that contains the changes that we discussed in https://github.com/apache/hive/pull/4252. 
   
   At current phase, it is a draft and I'm in the middle of porting the commits but I think it is the time to start a conversation about it. 
   
   First of all, I have a question about the namings: do you know why Iceberg and Hive has different naming for the class using for locking? In Hive, it is called `HiveCommitLog` and in Iceberg, it is `HiveMetastoreLock`. 
   
   Thanks, 
   Zsolt


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235908200


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/Catalogs.java:
##########
@@ -254,47 +251,21 @@ static Optional<Catalog> loadCatalog(Configuration conf, String catalogName) {
    * @param catalogType type of the catalog
    * @return complete map of catalog properties
    */
-  private static Map<String, String> getCatalogProperties(Configuration conf, String catalogName, String catalogType) {
+  private static Map<String, String> getCatalogProperties(
+      Configuration conf, String catalogName, String catalogType) {
     Map<String, String> catalogProperties = Maps.newHashMap();
     conf.forEach(config -> {
       if (config.getKey().startsWith(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX)) {
         catalogProperties.putIfAbsent(
-            config.getKey().substring(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX.length()),
-            config.getValue());
+                config.getKey().substring(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX.length()),
+                config.getValue());
       } else if (config.getKey().startsWith(InputFormatConfig.CATALOG_CONFIG_PREFIX + catalogName)) {

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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1238687479


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -176,11 +180,10 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
     CommitStatus commitStatus = CommitStatus.FAILURE;
     boolean updateHiveTable = false;
-    HiveCommitLock commitLock = null;
 
+    HiveLock lock = lockObject(metadata);

Review Comment:
   Fixed.



##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1240904422


##########
iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/SerializableTable.java:
##########
@@ -1,420 +0,0 @@
-/*
- * 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;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.iceberg.encryption.EncryptionManager;
-import org.apache.iceberg.hadoop.HadoopConfigurable;
-import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.io.LocationProvider;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.SerializableMap;
-import org.apache.iceberg.util.SerializableSupplier;
-
-/**
- * A read-only serializable table that can be sent to other nodes in a cluster.
- *
- * <p>An instance of this class represents an immutable serializable copy of a table state and will
- * not reflect any subsequent changed made to the original table.
- *
- * <p>While this class captures the metadata file location that can be used to load the complete
- * table metadata, it directly persists the current schema, spec, sort order, table properties to
- * avoid reading the metadata file from other nodes for frequently needed metadata.
- *
- * <p>The implementation assumes the passed instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} are serializable. If you are serializing the table using a custom
- * serialization framework like Kryo, those instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} must be supported by that particular serialization framework.
- *
- * <p><em>Note:</em> loading the complete metadata from a large number of nodes can overwhelm the
- * storage.
- */
-public class SerializableTable implements Table, Serializable {
-
-  private final String name;
-  private final String location;
-  private final String metadataFileLocation;
-  private final Map<String, String> properties;
-  private final String schemaAsJson;
-  private final int defaultSpecId;
-  private final Map<Integer, String> specAsJsonMap;
-  private final String sortOrderAsJson;
-  private final FileIO io;
-  private final EncryptionManager encryption;
-  private final LocationProvider locationProvider;
-  private final Map<String, SnapshotRef> refs;
-
-  private transient volatile Table lazyTable = null;
-  private transient volatile Schema lazySchema = null;
-  private transient volatile Map<Integer, PartitionSpec> lazySpecs = null;
-  private transient volatile SortOrder lazySortOrder = null;
-
-  protected SerializableTable(Table table) {
-    this.name = table.name();
-    this.location = table.location();
-    this.metadataFileLocation = metadataFileLocation(table);
-    this.properties = SerializableMap.copyOf(table.properties());
-    this.schemaAsJson = SchemaParser.toJson(table.schema());
-    this.defaultSpecId = table.spec().specId();
-    this.specAsJsonMap = Maps.newHashMap();
-    Map<Integer, PartitionSpec> specs = table.specs();
-    specs.forEach((specId, spec) -> specAsJsonMap.put(specId, PartitionSpecParser.toJson(spec)));
-    this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder());
-    this.io = fileIO(table);
-    this.encryption = table.encryption();
-    this.locationProvider = table.locationProvider();
-    this.refs = SerializableMap.copyOf(table.refs());
-  }
-
-  /**
-   * Creates a read-only serializable table that can be sent to other nodes in a cluster.
-   *
-   * @param table the original table to copy the state from
-   * @return a read-only serializable table reflecting the current state of the original table
-   */
-  public static Table copyOf(Table table) {
-    if (table instanceof BaseMetadataTable) {
-      return new SerializableMetadataTable((BaseMetadataTable) table);
-    } else {
-      return new SerializableTable(table);
-    }
-  }
-
-  private String metadataFileLocation(Table table) {
-    if (table instanceof HasTableOperations) {
-      TableOperations ops = ((HasTableOperations) table).operations();
-      return ops.current().metadataFileLocation();
-    } else {
-      return null;
-    }
-  }
-
-  private FileIO fileIO(Table table) {
-    if (table.io() instanceof HadoopConfigurable) {
-      ((HadoopConfigurable) table.io()).serializeConfWith(SerializableConfSupplier::new);
-    }
-
-    return table.io();
-  }
-
-  private Table lazyTable() {
-    if (lazyTable == null) {
-      synchronized (this) {
-        if (lazyTable == null) {
-          if (metadataFileLocation == null) {
-            throw new UnsupportedOperationException(
-                    "Cannot load metadata: metadata file location is null");
-          }
-
-          TableOperations ops =
-                  new StaticTableOperations(metadataFileLocation, io, locationProvider);
-          this.lazyTable = newTable(ops, name);
-        }
-      }
-    }
-
-    return lazyTable;
-  }
-
-  protected Table newTable(TableOperations ops, String tableName) {
-    return new BaseTable(ops, tableName);
-  }
-
-  @Override
-  public String name() {
-    return name;
-  }
-
-  @Override
-  public String location() {
-    return location;
-  }
-
-  @Override
-  public Map<String, String> properties() {
-    return properties;
-  }
-
-  @Override
-  public Schema schema() {
-    if (lazySchema == null) {
-      synchronized (this) {
-        if (lazySchema == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySchema = SchemaParser.fromJson(schemaAsJson);
-        } else if (lazySchema == null) {
-          this.lazySchema = lazyTable.schema();
-        }
-      }
-    }
-
-    return lazySchema;
-  }
-
-  @Override
-  public Map<Integer, Schema> schemas() {
-    return lazyTable().schemas();
-  }
-
-  @Override
-  public PartitionSpec spec() {
-    return specs().get(defaultSpecId);
-  }
-
-  @Override
-  public Map<Integer, PartitionSpec> specs() {
-    if (lazySpecs == null) {
-      synchronized (this) {
-        if (lazySpecs == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          Map<Integer, PartitionSpec> specs = Maps.newHashMapWithExpectedSize(specAsJsonMap.size());
-          specAsJsonMap.forEach(
-              (specId, specAsJson) -> {
-                specs.put(specId, PartitionSpecParser.fromJson(schema(), specAsJson));
-              });
-          this.lazySpecs = specs;
-        } else if (lazySpecs == null) {
-          this.lazySpecs = lazyTable.specs();
-        }
-      }
-    }
-
-    return lazySpecs;
-  }
-
-  @Override
-  public SortOrder sortOrder() {
-    if (lazySortOrder == null) {
-      synchronized (this) {
-        if (lazySortOrder == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySortOrder = SortOrderParser.fromJson(schema(), sortOrderAsJson);
-        } else if (lazySortOrder == null) {
-          this.lazySortOrder = lazyTable.sortOrder();
-        }
-      }
-    }
-
-    return lazySortOrder;
-  }
-
-  @Override
-  public Map<Integer, SortOrder> sortOrders() {
-    return lazyTable().sortOrders();
-  }
-
-  @Override
-  public FileIO io() {
-    return io;
-  }
-
-  @Override
-  public EncryptionManager encryption() {
-    return encryption;
-  }
-
-  @Override
-  public LocationProvider locationProvider() {
-    return locationProvider;
-  }
-
-  @Override
-  public List<StatisticsFile> statisticsFiles() {
-    return lazyTable().statisticsFiles();
-  }
-
-  @Override
-  public Map<String, SnapshotRef> refs() {
-    return refs;
-  }
-
-  @Override
-  public void refresh() {
-    throw new UnsupportedOperationException(errorMsg("refresh"));
-  }
-
-  @Override
-  public TableScan newScan() {
-    return lazyTable().newScan();
-  }
-
-  public IncrementalAppendScan newIncrementalAppendScan() {

Review Comment:
   Comparing the diff between the `patched-iceberg-core-SerializableTable` in hive and `org.apache.iceberg.SerializableTable` in iceberg .
   ![image](https://github.com/apache/hive/assets/26441677/89a7c068-3126-4717-8883-ee10b71ddb19)
   



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1234985615


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java:
##########
@@ -21,25 +21,61 @@
 
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
 import org.apache.thrift.TException;
+import org.immutables.value.Value;
 
+/**
+ * A ClientPool that caches the underlying HiveClientPool instances.
+ *
+ * <p>The following key elements are supported and can be specified via {@link
+ * CatalogProperties#CLIENT_POOL_CACHE_KEYS}:
+ *
+ * <ul>
+ *   <li>ugi - the Hadoop UserGroupInformation instance that represents the current user using the
+ *       cache.
+ *   <li>user_name - similar to UGI but only includes the user's name determined by
+ *       UserGroupInformation#getUserName.
+ *   <li>conf - name of an arbitrary configuration. The value of the configuration will be extracted
+ *       from catalog properties and added to the cache key. A conf element should start with a
+ *       "conf:" prefix which is followed by the configuration name. E.g. specifying "conf:a.b.c"
+ *       will add "a.b.c" to the key, and so that configurations with different default catalog
+ *       wouldn't share the same client pool. Multiple conf elements can be specified.
+ * </ul>
+ */
 public class CachedClientPool implements ClientPool<IMetaStoreClient, TException> {
 
-  private static Cache<String, HiveClientPool> clientPoolCache;
+  private static final String CONF_ELEMENT_PREFIX = "conf:";
+
+  private static Cache<Key, HiveClientPool> clientPoolCache;
 
   private final Configuration conf;
   private final String metastoreUri;

Review Comment:
   this one could be dropped. not used anywhere and also not present in iceberg



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ayushtkn commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "ayushtkn (via GitHub)" <gi...@apache.org>.
ayushtkn commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235143595


##########
iceberg/iceberg-catalog/pom.xml:
##########
@@ -74,5 +78,10 @@
       <artifactId>junit-vintage-engine</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>

Review Comment:
   why was this required? Which test fails without this? As of now it is inheriting from parent POM, Shouldn't we use     
   ```
   <iceberg.mockito-core.version>3.4.4</iceberg.mockito-core.version> 
   ```
   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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "SourabhBadhya (via GitHub)" <gi...@apache.org>.
SourabhBadhya commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235187827


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -364,7 +413,6 @@ private void setHmsTableParameters(String newMetadataLocation, Table tbl, TableM
     tbl.setParameters(parameters);
   }
 
-  @VisibleForTesting

Review Comment:
   Why is this annotation removed? I see the same in upstream Iceberg code.
   There are also many such instances in this file where the annotation is removed but I am generalising all of them 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235303799


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -176,11 +180,10 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) {
 
     CommitStatus commitStatus = CommitStatus.FAILURE;
     boolean updateHiveTable = false;
-    HiveCommitLock commitLock = null;
 
+    HiveLock lock = lockObject(metadata);

Review Comment:
   I added it in commit https://github.com/apache/hive/pull/4291/commits/484c5e764ec40adf7a579260ec73e9952eb60e91
   
   Thank you for finding 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235712051


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {

Review Comment:
   This was added when cherry-picking one of the commits for this upgrade: "Hive: Fix concurrent transactions overwriting commits by adding hive lock heartbeats. (#5036)". Not sure why it was added, @InvisibleProgrammer is in better position to answer that. But comparing to Iceberg repo, it is called from one more public method in hive-iceberg: `HiveIcebergMetaHook.preAlterTable`, maybe it is related to 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235812771


##########
iceberg/iceberg-catalog/pom.xml:
##########
@@ -74,5 +78,10 @@
       <artifactId>junit-vintage-engine</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>

Review Comment:
   Seems unneeded, the parent pom.xml has the same version as a dependency. 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235010227


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -510,6 +548,53 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
       return metadata.propertyAsBoolean(TableProperties.ENGINE_HIVE_ENABLED, false);
     }
 
-    return conf.getBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
+    return conf.getBoolean(
+        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) {
+    if (metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.HIVE_LOCK_ENABLED, false);
+    }
+
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.HIVE_LOCK_ENABLED_DEFAULT);
+  }
+
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      Optional<Long> txnId = Optional.empty();
+
+      SessionState sessionState = SessionState.get();
+      if (sessionState != null) {
+        HiveTxnManager txnMgr = sessionState.getTxnMgr();
+        if (txnMgr != null) {
+          txnId = Optional.of(txnMgr.getCurrentTxnId());
+        }
+      }
+
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName,   txnId);

Review Comment:
   could you please move txnId fetch logic in 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236822862


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -337,6 +342,23 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
     }
   }
 
+  @NotNull
+  private static Optional<Long> getTxnId() {
+    Optional<Long> txnId;
+    txnId = Optional.empty();
+
+    SessionState sessionState = SessionState.get();
+
+    if (sessionState != null) {
+      HiveTxnManager txnMgr = sessionState.getTxnMgr();
+      if (txnMgr != null) {
+        txnId = Optional.of(txnMgr.getCurrentTxnId());
+      }
+    }

Review Comment:
   As we have discussed with @deniskuzZ , I reverted the 'Pass txnid to MetaStoreLock' commit



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] simhadri-g commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "simhadri-g (via GitHub)" <gi...@apache.org>.
simhadri-g commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1610901803

   LGTM +1


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on pull request #4291: Hive-27306: port iceberg catalog changes

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

   > First of all, I have a question about the namings: do you know why Iceberg and Hive has different naming for the class using for locking? In Hive, it is called `HiveCommitLog` and in Iceberg, it is `HiveMetastoreLock`.
   
   It was after my time, but I expect that @szlta give up on getting his refactor into Iceberg, but needed it in Hive.


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1573173112

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1610361533

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235262220


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -510,6 +548,53 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
       return metadata.propertyAsBoolean(TableProperties.ENGINE_HIVE_ENABLED, false);
     }
 
-    return conf.getBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
+    return conf.getBoolean(
+        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) {
+    if (metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.HIVE_LOCK_ENABLED, false);
+    }
+
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.HIVE_LOCK_ENABLED_DEFAULT);
+  }
+
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      Optional<Long> txnId = Optional.empty();
+
+      SessionState sessionState = SessionState.get();
+      if (sessionState != null) {
+        HiveTxnManager txnMgr = sessionState.getTxnMgr();
+        if (txnMgr != null) {
+          txnId = Optional.of(txnMgr.getCurrentTxnId());
+        }
+      }
+
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName,   txnId);

Review Comment:
   sorry, i don't get it. why can't you fetch txnId inside of MetastoreLock constructor?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1599989817

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [69 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235102892


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {

Review Comment:
   why is that check here? it's already done inside `public void unlock() `



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235106964


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =

Review Comment:
   where do you set txnId?



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235176904


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreUtil.java:
##########
@@ -19,57 +19,61 @@
 
 package org.apache.iceberg.hive;
 
+import java.util.Map;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.iceberg.common.DynMethods;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 
 public class MetastoreUtil {
-
-  // this class is unique to Hive3 and cannot be found in Hive2, therefore a good proxy to see if
-  // we are working against Hive3 dependencies
-  private static final String HIVE3_UNIQUE_CLASS = "org.apache.hadoop.hive.serde2.io.DateWritableV2";
-
-  private static final DynMethods.UnboundMethod ALTER_TABLE = DynMethods.builder("alter_table")
-      .impl(IMetaStoreClient.class, "alter_table_with_environmentContext",
-          String.class, String.class, Table.class, EnvironmentContext.class)
-      .impl(IMetaStoreClient.class, "alter_table",
-          String.class, String.class, Table.class, EnvironmentContext.class)
-      .impl(IMetaStoreClient.class, "alter_table",
-          String.class, String.class, Table.class)
+  private static final DynMethods.UnboundMethod ALTER_TABLE =
+      DynMethods.builder("alter_table")
+          .impl(
+              IMetaStoreClient.class,
+              "alter_table_with_environmentContext",
+              String.class,
+              String.class,
+              Table.class,
+              EnvironmentContext.class)
+          .impl(
+              IMetaStoreClient.class,
+              "alter_table",
+              String.class,
+              String.class,
+              Table.class,
+              EnvironmentContext.class)
+          .impl(IMetaStoreClient.class, "alter_table", String.class, String.class, Table.class)
       .build();
 
-  private static final boolean HIVE3_PRESENT_ON_CLASSPATH = detectHive3();

Review Comment:
   why did we remove this? it's present in upstream iceberg



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] ayushtkn commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "ayushtkn (via GitHub)" <gi...@apache.org>.
ayushtkn commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235166922


##########
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java:
##########
@@ -332,10 +338,13 @@ public void testLoadCatalogLocation() {
   @Test
   public void testLoadCatalogUnknown() {
     String catalogName = "barCatalog";
-    conf.set(InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE), "fooType");
-    AssertHelpers.assertThrows(
-        "should complain about catalog not supported", UnsupportedOperationException.class,
-        "Unknown catalog type:", () -> Catalogs.loadCatalog(conf, catalogName));
+    conf.set(
+        InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE),
+        "fooType");

Review Comment:
   nit: this is just a formatting change, it could have stayed as is
   ```
       conf.set(InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE), "fooType");
   ```
   



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235326651


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -473,11 +512,10 @@ private void cleanupMetadataAndUnlock(CommitStatus commitStatus, String metadata
     }
   }
 
-  @VisibleForTesting
-  void doUnlock(HiveCommitLock lock) {
+  void doUnlock(HiveLock lock) {

Review Comment:
   I assume it is to make cleanupMetadataAndUnlock easier to read. 
   TBH, I can only guess the reason why it shallows the exception: both this and cleanupMetadataAndUnlock are used in finally blocks. I assume the author didn't want to add a new exception handling block inside the finally block. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235657911


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.

Review Comment:
   No, hive lock heartbeating was added in this commit Hive: Fix concurrent transactions overwriting commits by adding hive lock heartbeats. (#5036) and it was backported into hive-iceberg module.



##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.

Review Comment:
   No, hive lock heartbeating was added in this commit Hive: `Fix concurrent transactions overwriting commits by adding hive lock heartbeats. (#5036)` and it was backported into hive-iceberg module.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1243864151


##########
iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/SerializableTable.java:
##########
@@ -1,420 +0,0 @@
-/*
- * 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;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.iceberg.encryption.EncryptionManager;
-import org.apache.iceberg.hadoop.HadoopConfigurable;
-import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.io.LocationProvider;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.SerializableMap;
-import org.apache.iceberg.util.SerializableSupplier;
-
-/**
- * A read-only serializable table that can be sent to other nodes in a cluster.
- *
- * <p>An instance of this class represents an immutable serializable copy of a table state and will
- * not reflect any subsequent changed made to the original table.
- *
- * <p>While this class captures the metadata file location that can be used to load the complete
- * table metadata, it directly persists the current schema, spec, sort order, table properties to
- * avoid reading the metadata file from other nodes for frequently needed metadata.
- *
- * <p>The implementation assumes the passed instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} are serializable. If you are serializing the table using a custom
- * serialization framework like Kryo, those instances of {@link FileIO}, {@link EncryptionManager},
- * {@link LocationProvider} must be supported by that particular serialization framework.
- *
- * <p><em>Note:</em> loading the complete metadata from a large number of nodes can overwhelm the
- * storage.
- */
-public class SerializableTable implements Table, Serializable {
-
-  private final String name;
-  private final String location;
-  private final String metadataFileLocation;
-  private final Map<String, String> properties;
-  private final String schemaAsJson;
-  private final int defaultSpecId;
-  private final Map<Integer, String> specAsJsonMap;
-  private final String sortOrderAsJson;
-  private final FileIO io;
-  private final EncryptionManager encryption;
-  private final LocationProvider locationProvider;
-  private final Map<String, SnapshotRef> refs;
-
-  private transient volatile Table lazyTable = null;
-  private transient volatile Schema lazySchema = null;
-  private transient volatile Map<Integer, PartitionSpec> lazySpecs = null;
-  private transient volatile SortOrder lazySortOrder = null;
-
-  protected SerializableTable(Table table) {
-    this.name = table.name();
-    this.location = table.location();
-    this.metadataFileLocation = metadataFileLocation(table);
-    this.properties = SerializableMap.copyOf(table.properties());
-    this.schemaAsJson = SchemaParser.toJson(table.schema());
-    this.defaultSpecId = table.spec().specId();
-    this.specAsJsonMap = Maps.newHashMap();
-    Map<Integer, PartitionSpec> specs = table.specs();
-    specs.forEach((specId, spec) -> specAsJsonMap.put(specId, PartitionSpecParser.toJson(spec)));
-    this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder());
-    this.io = fileIO(table);
-    this.encryption = table.encryption();
-    this.locationProvider = table.locationProvider();
-    this.refs = SerializableMap.copyOf(table.refs());
-  }
-
-  /**
-   * Creates a read-only serializable table that can be sent to other nodes in a cluster.
-   *
-   * @param table the original table to copy the state from
-   * @return a read-only serializable table reflecting the current state of the original table
-   */
-  public static Table copyOf(Table table) {
-    if (table instanceof BaseMetadataTable) {
-      return new SerializableMetadataTable((BaseMetadataTable) table);
-    } else {
-      return new SerializableTable(table);
-    }
-  }
-
-  private String metadataFileLocation(Table table) {
-    if (table instanceof HasTableOperations) {
-      TableOperations ops = ((HasTableOperations) table).operations();
-      return ops.current().metadataFileLocation();
-    } else {
-      return null;
-    }
-  }
-
-  private FileIO fileIO(Table table) {
-    if (table.io() instanceof HadoopConfigurable) {
-      ((HadoopConfigurable) table.io()).serializeConfWith(SerializableConfSupplier::new);
-    }
-
-    return table.io();
-  }
-
-  private Table lazyTable() {
-    if (lazyTable == null) {
-      synchronized (this) {
-        if (lazyTable == null) {
-          if (metadataFileLocation == null) {
-            throw new UnsupportedOperationException(
-                    "Cannot load metadata: metadata file location is null");
-          }
-
-          TableOperations ops =
-                  new StaticTableOperations(metadataFileLocation, io, locationProvider);
-          this.lazyTable = newTable(ops, name);
-        }
-      }
-    }
-
-    return lazyTable;
-  }
-
-  protected Table newTable(TableOperations ops, String tableName) {
-    return new BaseTable(ops, tableName);
-  }
-
-  @Override
-  public String name() {
-    return name;
-  }
-
-  @Override
-  public String location() {
-    return location;
-  }
-
-  @Override
-  public Map<String, String> properties() {
-    return properties;
-  }
-
-  @Override
-  public Schema schema() {
-    if (lazySchema == null) {
-      synchronized (this) {
-        if (lazySchema == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySchema = SchemaParser.fromJson(schemaAsJson);
-        } else if (lazySchema == null) {
-          this.lazySchema = lazyTable.schema();
-        }
-      }
-    }
-
-    return lazySchema;
-  }
-
-  @Override
-  public Map<Integer, Schema> schemas() {
-    return lazyTable().schemas();
-  }
-
-  @Override
-  public PartitionSpec spec() {
-    return specs().get(defaultSpecId);
-  }
-
-  @Override
-  public Map<Integer, PartitionSpec> specs() {
-    if (lazySpecs == null) {
-      synchronized (this) {
-        if (lazySpecs == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          Map<Integer, PartitionSpec> specs = Maps.newHashMapWithExpectedSize(specAsJsonMap.size());
-          specAsJsonMap.forEach(
-              (specId, specAsJson) -> {
-                specs.put(specId, PartitionSpecParser.fromJson(schema(), specAsJson));
-              });
-          this.lazySpecs = specs;
-        } else if (lazySpecs == null) {
-          this.lazySpecs = lazyTable.specs();
-        }
-      }
-    }
-
-    return lazySpecs;
-  }
-
-  @Override
-  public SortOrder sortOrder() {
-    if (lazySortOrder == null) {
-      synchronized (this) {
-        if (lazySortOrder == null && lazyTable == null) {
-          // prefer parsing JSON as opposed to loading the metadata
-          this.lazySortOrder = SortOrderParser.fromJson(schema(), sortOrderAsJson);
-        } else if (lazySortOrder == null) {
-          this.lazySortOrder = lazyTable.sortOrder();
-        }
-      }
-    }
-
-    return lazySortOrder;
-  }
-
-  @Override
-  public Map<Integer, SortOrder> sortOrders() {
-    return lazyTable().sortOrders();
-  }
-
-  @Override
-  public FileIO io() {
-    return io;
-  }
-
-  @Override
-  public EncryptionManager encryption() {
-    return encryption;
-  }
-
-  @Override
-  public LocationProvider locationProvider() {
-    return locationProvider;
-  }
-
-  @Override
-  public List<StatisticsFile> statisticsFiles() {
-    return lazyTable().statisticsFiles();
-  }
-
-  @Override
-  public Map<String, SnapshotRef> refs() {
-    return refs;
-  }
-
-  @Override
-  public void refresh() {
-    throw new UnsupportedOperationException(errorMsg("refresh"));
-  }
-
-  @Override
-  public TableScan newScan() {
-    return lazyTable().newScan();
-  }
-
-  public IncrementalAppendScan newIncrementalAppendScan() {

Review Comment:
   Fixed.



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ merged pull request #4291: Hive-27306: port iceberg catalog changes

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


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1239589471


##########
iceberg/patched-iceberg-core/pom.xml:
##########
@@ -76,6 +76,8 @@
                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
                   <excludes>
                       **/HadoopInputFile.class
+                      **/TableProperties.class
+                      **/ConfigProperties.class

Review Comment:
   Thank you for spotting this. I did some clean up and removed `SerializableTable` as well. 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236784065


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java:
##########
@@ -510,6 +548,53 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c
       return metadata.propertyAsBoolean(TableProperties.ENGINE_HIVE_ENABLED, false);
     }
 
-    return conf.getBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT);
+    return conf.getBoolean(
+        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) {
+    if (metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) {
+      // We know that the property is set, so default value will not be used,
+      return metadata.propertyAsBoolean(TableProperties.HIVE_LOCK_ENABLED, false);
+    }
+
+    return conf.getBoolean(
+        ConfigProperties.LOCK_HIVE_ENABLED, TableProperties.HIVE_LOCK_ENABLED_DEFAULT);
+  }
+
+  @VisibleForTesting
+  HiveLock lockObject(TableMetadata metadata) {
+    if (hiveLockEnabled(metadata, conf)) {
+      Optional<Long> txnId = Optional.empty();
+
+      SessionState sessionState = SessionState.get();
+      if (sessionState != null) {
+        HiveTxnManager txnMgr = sessionState.getTxnMgr();
+        if (txnMgr != null) {
+          txnId = Optional.of(txnMgr.getCurrentTxnId());
+        }
+      }
+
+      return new MetastoreLock(conf, metaClients, catalogName, database, tableName,   txnId);

Review Comment:
   As we have discussed, I reverted the 'Pass txnid to MetaStoreLock' commit



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236821454


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {

Review Comment:
   It wasn't a porting mistake, it was waaay older. Anyway, I fixed 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1234992973


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:
##########
@@ -244,11 +252,16 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
   @Override
   public void createNamespace(Namespace namespace, Map<String, String> meta) {
     Preconditions.checkArgument(
-        !namespace.isEmpty(),
-        "Cannot create namespace with invalid name: %s", namespace);
-    Preconditions.checkArgument(isValidateNamespace(namespace),
-        "Cannot support multi part namespace in Hive Metastore: %s", namespace);
-
+        !namespace.isEmpty(), "Cannot create namespace with invalid name: %s", namespace);
+    Preconditions.checkArgument(
+        isValidateNamespace(namespace),
+        "Cannot support multi part namespace in Hive Metastore: %s",

Review Comment:
   Search in that class for "Metastore" has a few typos. Not in upstream iceberg



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235094493


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;

Review Comment:
   hmsLockId and txnId are different things !!!



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235141956


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:
##########
@@ -244,11 +252,16 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
   @Override
   public void createNamespace(Namespace namespace, Map<String, String> meta) {
     Preconditions.checkArgument(
-        !namespace.isEmpty(),
-        "Cannot create namespace with invalid name: %s", namespace);
-    Preconditions.checkArgument(isValidateNamespace(namespace),
-        "Cannot support multi part namespace in Hive Metastore: %s", namespace);
-
+        !namespace.isEmpty(), "Cannot create namespace with invalid name: %s", namespace);
+    Preconditions.checkArgument(
+        isValidateNamespace(namespace),
+        "Cannot support multi part namespace in Hive Metastore: %s",

Review Comment:
   Sorry, I don't get it. Could you please show me an example of the typo? 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235099028


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    metaClients.run(
+        client -> {
+          client.unlock(lockId);
+          return null;
+        });
+  }
+
+
+  private void acquireJvmLock() {
+    if (jvmLock != null) {
+      throw new IllegalStateException(String.format("Cannot call acquireLock twice for %s", fullName));
+    }
+
+    jvmLock = commitLockCache.get(fullName, t -> new ReentrantLock(true));
+    jvmLock.lock();
+  }
+
+  private void releaseJvmLock() {
+    if (jvmLock != null) {
+      jvmLock.unlock();
+      jvmLock = null;
+    }
+  }
+
+  private static void initTableLevelLockCache(long evictionTimeout) {
+    if (commitLockCache == null) {
+      synchronized (MetastoreLock.class) {
+        if (commitLockCache == null) {
+          commitLockCache =
+                  Caffeine.newBuilder()
+                          .expireAfterAccess(evictionTimeout, TimeUnit.MILLISECONDS)
+                          .build();
+        }
+      }
+    }
+  }
+
+  public String getTableName() {

Review Comment:
   is that 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "deniskuzZ (via GitHub)" <gi...@apache.org>.
deniskuzZ commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235260221


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java:
##########
@@ -21,25 +21,61 @@
 
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.ThreadPools;
 import org.apache.thrift.TException;
+import org.immutables.value.Value;
 
+/**
+ * A ClientPool that caches the underlying HiveClientPool instances.
+ *
+ * <p>The following key elements are supported and can be specified via {@link
+ * CatalogProperties#CLIENT_POOL_CACHE_KEYS}:
+ *
+ * <ul>
+ *   <li>ugi - the Hadoop UserGroupInformation instance that represents the current user using the
+ *       cache.
+ *   <li>user_name - similar to UGI but only includes the user's name determined by
+ *       UserGroupInformation#getUserName.
+ *   <li>conf - name of an arbitrary configuration. The value of the configuration will be extracted
+ *       from catalog properties and added to the cache key. A conf element should start with a
+ *       "conf:" prefix which is followed by the configuration name. E.g. specifying "conf:a.b.c"
+ *       will add "a.b.c" to the key, and so that configurations with different default catalog
+ *       wouldn't share the same client pool. Multiple conf elements can be specified.
+ * </ul>
+ */
 public class CachedClientPool implements ClientPool<IMetaStoreClient, TException> {
 
-  private static Cache<String, HiveClientPool> clientPoolCache;
+  private static final String CONF_ELEMENT_PREFIX = "conf:";
+
+  private static Cache<Key, HiveClientPool> clientPoolCache;
 
   private final Configuration conf;
   private final String metastoreUri;

Review Comment:
   no, metastoreUri



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] difin commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "difin (via GitHub)" <gi...@apache.org>.
difin commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1235712051


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java:
##########
@@ -0,0 +1,542 @@
+/*
+ * 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.annotations.VisibleForTesting;
+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 static volatile Cache<String, ReentrantLock> commitLockCache;
+
+  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 Optional<Long> hmsLockId = Optional.empty();
+  private ReentrantLock jvmLock = null;
+  private Heartbeat heartbeat = null;
+
+  public MetastoreLock(Configuration conf, ClientPool<IMetaStoreClient, TException> metaClients,
+                       String catalogName, String databaseName, String tableName, Optional<Long> txnId) {
+    this.metaClients = metaClients;
+    this.fullName = catalogName + "." + databaseName + "." + tableName;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.hmsLockId = txnId;
+
+    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 HMS lock acquisition requests
+    acquireJvmLock();
+    hmsLockId = Optional.of(acquireLock());
+
+    // Starting heartbeat for the HMS lock
+    heartbeat =
+            new Heartbeat(metaClients, hmsLockId.get(), lockHeartbeatIntervalTime);
+    heartbeat.schedule(exitingScheduledExecutorService);
+  }
+
+  @Override
+  public void ensureActive() throws LockException {
+    if (heartbeat == null) {
+      throw new LockException("Lock is not active");
+    }
+
+    if (heartbeat.encounteredException != null) {
+      throw new LockException(
+              heartbeat.encounteredException,
+              "Failed to heartbeat for hive lock. %s",
+              heartbeat.encounteredException.getMessage());
+    }
+    if (!heartbeat.active()) {
+      throw new LockException("Hive lock heartbeat thread not active");
+    }
+  }
+
+  @Override
+  public void unlock() {
+    if (heartbeat != null) {
+      heartbeat.cancel();
+      exitingScheduledExecutorService.shutdown();
+    }
+
+    try {
+      unlock(hmsLockId);
+    } finally {
+      releaseJvmLock();
+    }
+  }
+
+  // TODO add lock heart beating for cases where default lock timeout is too low.
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private long acquireLock() throws LockException {
+    LockInfo lockInfo = createLock();
+
+    final long start = System.currentTimeMillis();
+    long duration = 0;
+    boolean timeout = false;
+    TException thriftError = 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) {
+      thriftError = e;
+    } finally {
+      if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockInfo.lockId));
+      }
+    }
+
+    if (!lockInfo.lockState.equals(LockState.ACQUIRED)) {
+      // 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);
+      }
+
+      if (thriftError != null) {
+        throw new LockException(
+                thriftError, "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);
+    } else {
+      return lockInfo.lockId;
+    }
+  }
+
+  /**
+   * Creates a lock, retrying if possible on failure.
+   *
+   * @return The {@link LockInfo} object for the successfully 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 createLock() throws LockException {
+    LockInfo lockInfo = new LockInfo();
+
+    String hostName;
+    try {
+      hostName = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException uhe) {
+      throw new LockException(uhe, "Error generating host name");
+    }
+
+    LockComponent lockComponent =
+            new LockComponent(LockType.EXCL_WRITE, LockLevel.TABLE, databaseName);
+    lockComponent.setTablename(tableName);
+    LockRequest lockRequest =
+            new LockRequest(
+                    Lists.newArrayList(lockComponent),
+                    HiveHadoopUtil.currentUser(),
+                    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() && e instanceof LockException &&
+                    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 create 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();
+                        if (lockFound != null) {
+                          lockInfo.lockId = lockFound.lockId;
+                          lockInfo.lockState = lockFound.lockState;
+                          LOG.info("Found lock {} by agentInfo {}", lockInfo, agentInfo);
+                          return;
+                        }
+                      }
+
+                      throw new LockException("Failed to find lock for table %s.%s", databaseName, tableName);
+                    } catch (InterruptedException e) {
+                      Thread.currentThread().interrupt();
+                      interrupted.set(true);
+                      LOG.warn(
+                              "Interrupted while trying to find lock for table {}.{}", databaseName, tableName, e);
+                      throw new LockException(
+                              e, "Interrupted while trying to find lock for table %s.%s", databaseName, tableName);
+                    }
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    interrupted.set(true);
+                    LOG.warn("Interrupted while creating lock on table {}.{}", databaseName, tableName, e);
+                    throw new LockException(
+                            e, "Interrupted while creating lock on table %s.%s", databaseName, tableName);
+                  }
+                },
+                LockException.class);
+
+    // This should be initialized always, or exception should be thrown.
+    LOG.debug("Lock {} created for table {}.{}", lockInfo, databaseName, 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.
+   *
+   * @return The {@link LockInfo} for the found lock, or <code>null</code> if nothing found
+   */
+  private LockInfo findLock() throws LockException, 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(databaseName);
+    showLocksRequest.setTablename(tableName);
+    ShowLocksResponse response;
+    try {
+      response = metaClients.run(client -> client.showLocks(showLocksRequest));
+    } catch (TException e) {
+      throw new LockException(e, "Failed to find lock for table %s.%s", databaseName, tableName);
+    }
+    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 void unlock(Optional<Long> lockId) {
+
+    Long id = null;
+    try {
+      if (!lockId.isPresent()) {
+        // Try to find the lock based on agentInfo. Only works with Hive 2 or later.
+        if (HiveVersion.min(HiveVersion.HIVE_2)) {
+          LockInfo lockInfo = findLock();
+          if (lockInfo == null) {
+            // No lock found
+            LOG.info("No lock found with {} agentInfo", agentInfo);
+            return;
+          }
+
+          id = lockInfo.lockId;
+        } else {
+          LOG.warn("Could not find lock with HMSClient {}", HiveVersion.current());
+          return;
+        }
+      } else {
+        id = lockId.get();
+      }
+
+      doUnlock(id);
+
+    } catch (InterruptedException ie) {
+      if (id != null) {
+        // Interrupted unlock. We try to unlock one more time if we have a lockId
+        try {
+          Thread.interrupted(); // Clear the interrupt status flag for now, so we can retry unlock
+          LOG.warn("Interrupted unlock we try one more time {}.{}", databaseName, tableName, ie);
+          doUnlock(id);
+        } catch (Exception e) {
+          LOG.warn("Failed to unlock even on 2nd attempt {}.{}", databaseName, tableName, e);
+        } finally {
+          Thread.currentThread().interrupt(); // Set back the interrupt status
+        }
+      } else {
+        Thread.currentThread().interrupt(); // Set back the interrupt status
+        LOG.warn("Interrupted finding locks to unlock {}.{}", databaseName, tableName, ie);
+      }
+    } catch (Exception e) {
+      LOG.warn("Failed to unlock {}.{}", databaseName, tableName, e);
+    }
+  }
+
+  @VisibleForTesting
+  void doUnlock(long lockId) throws TException, InterruptedException {
+    if (heartbeat != null) {

Review Comment:
   This was added when cherry-picking one of the commits for this upgrade: `Hive: Fix concurrent transactions overwriting commits by adding hive lock heartbeats. (#5036)`. Not sure why it was added, @InvisibleProgrammer is in better position to answer that. But comparing to Iceberg repo, it is called from one more public method in hive-iceberg: `HiveIcebergMetaHook.preAlterTable`, maybe it is related to 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1567371353

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1572682598

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1574394463

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [17 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [45 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236781578


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -324,19 +326,39 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           context.getProperties().get(OLD_TABLE_NAME)).toString());
     }
     if (commitLock == null) {
-      commitLock = new HiveCommitLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
-          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName());
+
+      Optional<Long> txnId = getTxnId();
+
+      commitLock = new MetastoreLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),

Review Comment:
   We discussed it in person. I reverted the 'Pass txnid to MetaStoreLock' commit



##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java:
##########
@@ -324,19 +326,39 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           context.getProperties().get(OLD_TABLE_NAME)).toString());
     }
     if (commitLock == null) {
-      commitLock = new HiveCommitLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
-          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName());
+
+      Optional<Long> txnId = getTxnId();
+
+      commitLock = new MetastoreLock(conf, new CachedClientPool(conf, Maps.fromProperties(catalogProperties)),
+          catalogProperties.getProperty(Catalogs.NAME), hmsTable.getDbName(), hmsTable.getTableName(), txnId);
     }
 
     try {
-      commitLock.acquire();
+      commitLock.lock();
       doPreAlterTable(hmsTable, context);
     } catch (Exception e) {
-      commitLock.release();
+      commitLock.unlock();
       throw new MetaException(StringUtils.stringifyException(e));
     }
   }
 
+  @NotNull
+  private static Optional<Long> getTxnId() {

Review Comment:
   We discussed it in person. I reverted the 'Pass txnid to MetaStoreLock' commit



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] InvisibleProgrammer commented on a diff in pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "InvisibleProgrammer (via GitHub)" <gi...@apache.org>.
InvisibleProgrammer commented on code in PR #4291:
URL: https://github.com/apache/hive/pull/4291#discussion_r1236824795


##########
iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java:
##########
@@ -244,11 +252,16 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) {
   @Override
   public void createNamespace(Namespace namespace, Map<String, String> meta) {
     Preconditions.checkArgument(
-        !namespace.isEmpty(),
-        "Cannot create namespace with invalid name: %s", namespace);
-    Preconditions.checkArgument(isValidateNamespace(namespace),
-        "Cannot support multi part namespace in Hive Metastore: %s", namespace);
-
+        !namespace.isEmpty(), "Cannot create namespace with invalid name: %s", namespace);
+    Preconditions.checkArgument(
+        isValidateNamespace(namespace),
+        "Cannot support multi part namespace in Hive Metastore: %s",

Review Comment:
   It was waaay older than this port. Anyway, I fixed 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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1601704682

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1536886912

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![B](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/B-16px.png 'B')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [30 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1610631703

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] sonarcloud[bot] commented on pull request #4291: Hive-27306: port iceberg catalog changes

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #4291:
URL: https://github.com/apache/hive/pull/4291#issuecomment-1604293492

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_hive&pullRequest=4291)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG) [8 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4291&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL) [75 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4291&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_hive&pullRequest=4291&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org