You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "veghlaci05 (via GitHub)" <gi...@apache.org> on 2023/03/03 18:33:17 UTC

[GitHub] [hive] veghlaci05 opened a new pull request, #4096: HIVE-27117: Fix compaction related flaky tests

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

   ### What changes were proposed in this pull request?
   Fix the flaky compaction tests
   
   ### Why are the changes needed?
   In order to make CI builds more stable
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Manually


-- 
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] abstractdog commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java:
##########
@@ -3647,9 +3647,12 @@ long generateCompactionQueueId(Statement stmt) throws SQLException, MetaExceptio
             + "no record found in next_compaction_queue_id");
       }
       long id = rs.getLong(1);
-      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1);
+      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id;
       LOG.debug("Going to execute update <{}>", s);
-      stmt.executeUpdate(s);
+      if (stmt.executeUpdate(s) != 1) {

Review Comment:
   okay, we can live with a follow-up task, but in this case, I would like to see further minor improvements here:
   - HIVE-27121 <- refer to the follow-up task for better reference in a code comment
   - use LOG.info at least, this is an unhappy codepath, debug level is not enough
   - show the current id in the log message for better reference
   -



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java:
##########
@@ -3647,9 +3647,12 @@ long generateCompactionQueueId(Statement stmt) throws SQLException, MetaExceptio
             + "no record found in next_compaction_queue_id");
       }
       long id = rs.getLong(1);
-      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1);
+      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id;
       LOG.debug("Going to execute update <{}>", s);
-      stmt.executeUpdate(s);
+      if (stmt.executeUpdate(s) != 1) {

Review Comment:
   okay, we can live with a follow-up task, but in this case, I would like to see further minor improvements here:
   - HIVE-27121 <- refer to the follow-up task for better reference in a code comment
   - use LOG.info at least, this is an unhappy codepath, debug level is not enough
   - show the current id in the log message for better reference



-- 
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] veghlaci05 merged pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


-- 
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 #4096: HIVE-27117: Fix compaction related flaky tests

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

   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=4096)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&resolved=false&types=BUG) [![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=4096&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&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] abstractdog commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorOnTezTest.java:
##########
@@ -185,23 +187,32 @@ protected void verifySuccessfulCompaction(int expectedSuccessfulCompactions) thr
   }
 
   protected HiveHookEvents.HiveHookEventProto getRelatedTezEvent(String dbTableName) throws Exception {
-    List<ProtoMessageReader<HiveHookEvents.HiveHookEventProto>> readers = TestHiveProtoLoggingHook.getTestReader(conf, tmpFolder);
-    for (ProtoMessageReader<HiveHookEvents.HiveHookEventProto> reader : readers) {
-      HiveHookEvents.HiveHookEventProto event = reader.readEvent();
-      boolean getRelatedEvent = false;
-      while (!getRelatedEvent) {
-        while (event != null && ExecutionMode.TEZ != ExecutionMode.valueOf(event.getExecutionMode())) {
-          event = reader.readEvent();
+    int retryCount = 3;
+    while (retryCount-- > 0) {
+      try {
+        List<ProtoMessageReader<HiveHookEvents.HiveHookEventProto>> readers = TestHiveProtoLoggingHook.getTestReader(conf, tmpFolder);

Review Comment:
   is it possible to write a countdownlatch magic here similarly to the fix in TestTxnCommands2



##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorOnTezTest.java:
##########
@@ -185,23 +187,32 @@ protected void verifySuccessfulCompaction(int expectedSuccessfulCompactions) thr
   }
 
   protected HiveHookEvents.HiveHookEventProto getRelatedTezEvent(String dbTableName) throws Exception {
-    List<ProtoMessageReader<HiveHookEvents.HiveHookEventProto>> readers = TestHiveProtoLoggingHook.getTestReader(conf, tmpFolder);
-    for (ProtoMessageReader<HiveHookEvents.HiveHookEventProto> reader : readers) {
-      HiveHookEvents.HiveHookEventProto event = reader.readEvent();
-      boolean getRelatedEvent = false;
-      while (!getRelatedEvent) {
-        while (event != null && ExecutionMode.TEZ != ExecutionMode.valueOf(event.getExecutionMode())) {
-          event = reader.readEvent();
+    int retryCount = 3;
+    while (retryCount-- > 0) {
+      try {
+        List<ProtoMessageReader<HiveHookEvents.HiveHookEventProto>> readers = TestHiveProtoLoggingHook.getTestReader(conf, tmpFolder);

Review Comment:
   is it possible to write a countdownlatch magic here similarly to the fix in TestTxnCommands2?



-- 
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] abstractdog commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -577,7 +577,7 @@ public RawStore getMS() throws MetaException {
     return getMSForConf(conf);
   }
 
-  public static RawStore getMSForConf(Configuration conf) throws MetaException {
+  public static synchronized RawStore getMSForConf(Configuration conf) throws MetaException {

Review Comment:
   can you please clarify how is this synchronization related to the solution?



-- 
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] abstractdog commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java:
##########
@@ -3647,9 +3647,12 @@ long generateCompactionQueueId(Statement stmt) throws SQLException, MetaExceptio
             + "no record found in next_compaction_queue_id");
       }
       long id = rs.getLong(1);
-      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1);
+      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id;
       LOG.debug("Going to execute update <{}>", s);
-      stmt.executeUpdate(s);
+      if (stmt.executeUpdate(s) != 1) {

Review Comment:
   this looks like a hack in production code while we're targeting test fixes here, which is a bit concerning
   this method is already very confusing: it looks like we cannot guarantee synchronization regarding NCQ_NEXT (which is okay if we have more instances running this code, right?), as we're selecting it, doing something here in java code (increment), then trying to write it back...isn't there a way to use AUTOINCREMENTed value across the supported RDBMS types?
   I mean, this recursive call is not something I really like, we should find a better synchronization pattern, I think a compaction queue id increment is not something that happens 100 times per second, so even a RDBMS-level lock is fine on this table I guess (I don't know this area though)
   
   don't get me wrong, I'm more than happy to see stability fixes, but simply retrying in production code reminds me of a good old pattern in hive code: "we have no idea what's going on, so let's retry :) "



-- 
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 #4096: HIVE-27117: Fix compaction related flaky tests

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

   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=4096)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&resolved=false&types=BUG) [![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=4096&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&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] abstractdog commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java:
##########
@@ -1434,6 +1434,8 @@ public void testTruncatePartitionAndCompactionConcurrent() throws Exception {
     execDDLOpAndCompactionConcurrently("TRUNCATE_PARTITION", true);
   }
   private void execDDLOpAndCompactionConcurrently(String opType, boolean isPartioned) throws Exception {
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_COMPACTOR_GATHER_STATS, false);

Review Comment:
   please comment on why is this needed and then it can be merged +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] veghlaci05 commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorOnTezTest.java:
##########
@@ -185,23 +187,32 @@ protected void verifySuccessfulCompaction(int expectedSuccessfulCompactions) thr
   }
 
   protected HiveHookEvents.HiveHookEventProto getRelatedTezEvent(String dbTableName) throws Exception {
-    List<ProtoMessageReader<HiveHookEvents.HiveHookEventProto>> readers = TestHiveProtoLoggingHook.getTestReader(conf, tmpFolder);
-    for (ProtoMessageReader<HiveHookEvents.HiveHookEventProto> reader : readers) {
-      HiveHookEvents.HiveHookEventProto event = reader.readEvent();
-      boolean getRelatedEvent = false;
-      while (!getRelatedEvent) {
-        while (event != null && ExecutionMode.TEZ != ExecutionMode.valueOf(event.getExecutionMode())) {
-          event = reader.readEvent();
+    int retryCount = 3;
+    while (retryCount-- > 0) {
+      try {
+        List<ProtoMessageReader<HiveHookEvents.HiveHookEventProto>> readers = TestHiveProtoLoggingHook.getTestReader(conf, tmpFolder);

Review Comment:
   Unofrtunately not possible, the problem here that I cannot mock the Event processing part to  release the latch.



-- 
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] abstractdog commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java:
##########
@@ -3647,9 +3647,12 @@ long generateCompactionQueueId(Statement stmt) throws SQLException, MetaExceptio
             + "no record found in next_compaction_queue_id");
       }
       long id = rs.getLong(1);
-      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1);
+      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id;
       LOG.debug("Going to execute update <{}>", s);
-      stmt.executeUpdate(s);
+      if (stmt.executeUpdate(s) != 1) {

Review Comment:
   this looks like a hack in production code while we're targeting test fixes here, which is a bit concerning
   this method is already very confusing: it looks like cannot guarantee synchronization regarding NCQ_NEXT (which is okay if we have more instances running this code, right?), as we're selecting it, doing something here in java code (increment), then trying to write it back, isn't there a way to use AUTOINCREMENTed value across the supported RDBMS types?
   I mean, this recursive call is not something I really like, we should find a better synchronization pattern, I think a compaction queue id increment is not something that happens 100 times per second, so even a RDBMS-level lock is fine on this table I guess (I don't know this area though)



-- 
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 #4096: HIVE-27117: Fix compaction related flaky tests

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

   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=4096)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&resolved=false&types=BUG) [![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=4096&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&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 #4096: HIVE-27117: Fix compaction related flaky tests

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

   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=4096)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&resolved=false&types=BUG) [![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=4096&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4096&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=4096&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=4096&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4096&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=4096&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=4096&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] abstractdog commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java:
##########
@@ -3647,9 +3647,12 @@ long generateCompactionQueueId(Statement stmt) throws SQLException, MetaExceptio
             + "no record found in next_compaction_queue_id");
       }
       long id = rs.getLong(1);
-      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1);
+      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id;
       LOG.debug("Going to execute update <{}>", s);
-      stmt.executeUpdate(s);
+      if (stmt.executeUpdate(s) != 1) {

Review Comment:
   this looks like a hack in production code while we're targeting test fixes here, which is a bit concerning
   this method is already very confusing: it looks like we cannot guarantee synchronization regarding NCQ_NEXT (which is okay if we have more instances running this code, right?), as we're selecting it, doing something here in java code (increment), then trying to write it back, isn't there a way to use AUTOINCREMENTed value across the supported RDBMS types?
   I mean, this recursive call is not something I really like, we should find a better synchronization pattern, I think a compaction queue id increment is not something that happens 100 times per second, so even a RDBMS-level lock is fine on this table I guess (I don't know this area though)
   
   don't get me wrong, I'm more than happy to see stability fixes, but simply retrying in production code reminds me of a good old pattern in hive code: "we have no idea what's going on, so let's retry :) "



-- 
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] veghlaci05 commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java:
##########
@@ -3647,9 +3647,12 @@ long generateCompactionQueueId(Statement stmt) throws SQLException, MetaExceptio
             + "no record found in next_compaction_queue_id");
       }
       long id = rs.getLong(1);
-      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1);
+      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id;
       LOG.debug("Going to execute update <{}>", s);
-      stmt.executeUpdate(s);
+      if (stmt.executeUpdate(s) != 1) {

Review Comment:
   This is a short term quick and dirty fix, as the id generation for compaction requests is broken. The old approach did not guarantee a unique id at all, and since Initiator processes the candidates in a parallel way, this could be a production issue too. Actually I have seen this duplicate key error in production logs before. The long term solution is to modify the id to auto increment field. It is already done this way for TXNS table. However, this change will be bigger, therefore I created a separate task for it: https://issues.apache.org/jira/browse/HIVE-27121



-- 
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] veghlaci05 commented on a diff in pull request #4096: HIVE-27117: Fix compaction related flaky tests

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java:
##########
@@ -577,7 +577,7 @@ public RawStore getMS() throws MetaException {
     return getMSForConf(conf);
   }
 
-  public static RawStore getMSForConf(Configuration conf) throws MetaException {
+  public static synchronized RawStore getMSForConf(Configuration conf) throws MetaException {

Review Comment:
   This is another flakyness. Without this, org.apache.hadoop.hive.ql.txn.compactor.TestCompactionMetrics.testInitiatorFailuresCountedCorrectly deadlocked repeatedly on my machine. 4 threads were trying to init the DB (and get the RawStore) concurrently at the smae time which resulted in a deadlock. 



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