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

[GitHub] [hive] SourabhBadhya opened a new pull request, #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   <!--
   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.
   -->
   As described in the parent task, once the cleaner is separated into different entities, implement a separate handler which can create requests for aborted transactions cleanup. This would move the aborted transaction cleanup exclusively to the cleaner.
   
   ### 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.
   -->
   Improvement in abort cleanup
   
   ### 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'.
   -->
   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] deniskuzZ commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;

Review Comment:
   could we inline var declaration and assignment?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java:
##########
@@ -516,6 +516,19 @@ Set<CompactionInfo> findPotentialCompactions(int abortedThreshold, long abortedT
   @RetrySemantics.ReadOnly
   List<CompactionInfo> findReadyToClean(long minOpenTxnWaterMark, long retentionTime) throws MetaException;
 
+  /**
+   * Find the aborted entries in TXN_COMPONENTS which can be used to
+   * clean directories belonging to transactions in aborted state.
+   * @param abortedTimeThreshold Age of table/partition's oldest aborted transaction involving a given table
+   *                            or partition that will trigger cleanup.
+   * @param abortedThreshold Number of aborted transactions involving a given table or partition
+   *                         that will trigger cleanup.
+   * @return Information of potential abort items that needs to be cleaned.
+   * @throws MetaException
+   */
+  @RetrySemantics.ReadOnly
+  List<AcidTxnInfo> findReadyToCleanForAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException;

Review Comment:
   maybe `findReadyToCleanAborts` ?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java:
##########
@@ -259,49 +247,11 @@ private void cleanUsingAcidDir(CompactionInfo ci, String location, long minOpenT
      */
 
     // Creating 'reader' list since we are interested in the set of 'obsolete' files
-    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(ci, validTxnList);
-    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
-
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-
-    // Collect all the files/dirs
-    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, path);
-    AcidDirectory dir = AcidUtils.getAcidState(fs, path, conf, validWriteIdList, Ref.from(false), false,
-            dirSnapshots);
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdListForCompactionCleaner(ci, validTxnList);
     Table table = metadataCache.computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.tableName));
-    boolean isDynPartAbort = CompactorUtil.isDynPartAbort(table, ci.partName);
-
-    List<Path> obsoleteDirs = CompactorUtil.getObsoleteDirs(dir, isDynPartAbort);
-    if (isDynPartAbort || dir.hasUncompactedAborts()) {
-      ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
-    }
-
-    List<Path> deleted = fsRemover.clean(new CleanupRequestBuilder().setLocation(location)
-            .setDbName(ci.dbname).setFullPartitionName(ci.getFullPartitionName())
-            .setRunAs(ci.runAs).setObsoleteDirs(obsoleteDirs).setPurge(true)
-            .build());
-
-    if (!deleted.isEmpty()) {
-      AcidMetricService.updateMetricsFromCleaner(ci.dbname, ci.tableName, ci.partName, dir.getObsolete(), conf,
-              txnHandler);
-    }
-
-    // Make sure there are no leftovers below the compacted watermark
-    boolean success = false;
-    conf.set(ValidTxnList.VALID_TXNS_KEY, new ValidReadTxnList().toString());
-    dir = AcidUtils.getAcidState(fs, path, conf, new ValidReaderWriteIdList(
-                    ci.getFullTableName(), new long[0], new BitSet(), ci.highestWriteId, Long.MAX_VALUE),
-            Ref.from(false), false, dirSnapshots);
+    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
 
-    List<Path> remained = subtract(CompactorUtil.getObsoleteDirs(dir, isDynPartAbort), deleted);
-    if (!remained.isEmpty()) {
-      LOG.warn("{} Remained {} obsolete directories from {}. {}",
-              idWatermark(ci), remained.size(), location, CompactorUtil.getDebugInfo(remained));
-    } else {
-      LOG.debug("{} All cleared below the watermark: {} from {}", idWatermark(ci), ci.highestWriteId, location);
-      success = true;
-    }
+    boolean success = cleanAndVerifyObsoleteDirectories(ci, location, validWriteIdList, table);

Review Comment:
   This is part of the cleanup task and will be removed when the config is removed.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandlerFactory.java:
##########
@@ -43,7 +44,14 @@ private TaskHandlerFactory() {
 
   public List<TaskHandler> getHandlers(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
                                                   boolean metricsEnabled, FSRemover fsRemover) {
-    return Arrays.asList(new CompactionCleaner(conf, txnHandler, metadataCache,
+    boolean useAbortHandler = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER);
+    List<TaskHandler> taskHandlers = new ArrayList<>();
+    if (useAbortHandler) {

Review Comment:
   This is part of the cleanup task when the config will be removed. Until then this is supposed to be here.



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidTxnInfo.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.hadoop.hive.metastore.txn;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
+import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+
+import java.util.Set;
+
+/**
+ * A class used for encapsulating information of abort-cleanup activities and compaction activities.
+ */
+public class AcidTxnInfo {

Review Comment:
   Removed. 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] sonarcloud[bot] commented on pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [27 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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] akshat0395 commented on pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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] sonarcloud[bot] commented on pull request #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [26 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorBase.java:
##########
@@ -89,6 +89,7 @@ public void setup() throws Exception {
     hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false);
     MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true);
     MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true);
+    MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, false);

Review Comment:
   why do we need this config? let's keep it simple



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -60,20 +60,20 @@
 public class TxnUtils {
   private static final Logger LOG = LoggerFactory.getLogger(TxnUtils.class);
 
-  public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns, long minOpenTxnGLB) {
-    long highWaterMark = minOpenTxnGLB - 1;
+  public static ValidTxnList createValidTxnListForCompactionCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
     long[] abortedTxns = new long[txns.getOpen_txnsSize()];
     BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
     int i = 0;
     for(long txnId : txns.getOpen_txns()) {
-      if(txnId > highWaterMark) {
+      if(txnId > highWatermark) {
         break;
       }
       if(abortedBits.get(i)) {

Review Comment:
   space



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandlerFactory.java:
##########
@@ -43,7 +44,14 @@ private TaskHandlerFactory() {
 
   public List<TaskHandler> getHandlers(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
                                                   boolean metricsEnabled, FSRemover fsRemover) {
-    return Arrays.asList(new CompactionCleaner(conf, txnHandler, metadataCache,
+    boolean useAbortHandler = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER);
+    List<TaskHandler> taskHandlers = new ArrayList<>();
+    if (useAbortHandler) {

Review Comment:
   no need for that check, from now on use Cleaner to handle aborts



-- 
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] TuroczyX commented on pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   @deniskuzZ Could you please also take time on 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] SourabhBadhya commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorWithAbortCleanupUsingCompactionCycle.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCompactor;
+import org.junit.Before;
+
+public class TestCompactorWithAbortCleanupUsingCompactionCycle extends TestCompactor {

Review Comment:
   This test class will be removed once it's known that this feature is stable. A separate task will be created for removing this use case.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -61,12 +60,10 @@ public void init(AtomicBoolean stop) throws Exception {
     cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
             conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_THREADS_NUM),
             COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
-    if (CollectionUtils.isEmpty(cleanupHandlers)) {
-      FSRemover fsRemover = new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache);
-      cleanupHandlers = TaskHandlerFactory.getInstance()
-              .getHandlers(conf, txnHandler, metadataCache,
-                      metricsEnabled, fsRemover);
-    }
+    FSRemover fsRemover = new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache);
+    cleanupHandlers = TaskHandlerFactory.getInstance()
+            .getHandlers(conf, txnHandler, metadataCache,
+                    metricsEnabled, fsRemover);

Review Comment:
   Done



##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java:
##########
@@ -649,6 +649,10 @@ public enum ConfVars {
     COMPACTOR_CLEANER_TABLECACHE_ON("metastore.compactor.cleaner.tablecache.on",
             "hive.compactor.cleaner.tablecache.on", true,
             "Enable table caching in the cleaner. Currently the cache is cleaned after each cycle."),
+    COMPACTOR_CLEAN_ABORTS_USING_CLEANER("metastore.compactor.clean.aborts.using.cleaner", "hive.compactor.clean.aborts.using.cleaner", true,

Review Comment:
   The plan is to keep this config till some point until we know that abort cleanup is stable. If there are any issues with the given handler, we can use the compaction cycle. I will create a task once its determined that this feature is stable and we can remove this feature flag and associated logic as well.



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java:
##########
@@ -162,31 +162,33 @@ public Set<CompactionInfo> findPotentialCompactions(int abortedThreshold,
         }
         rs.close();
 
-        // Check for aborted txns: number of aborted txns past threshold and age of aborted txns
-        // past time threshold
-        boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0;
-        String sCheckAborted = "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " +
-          "MIN(\"TXN_STARTED\"), COUNT(*) FROM \"TXNS\", \"TXN_COMPONENTS\" " +
-          "   WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + " " +
-          "GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " +
-              (checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold);
-
-        LOG.debug("Going to execute query <{}>", sCheckAborted);
-        rs = stmt.executeQuery(sCheckAborted);
-        long systemTime = System.currentTimeMillis();
-        while (rs.next()) {
-          boolean pastTimeThreshold =
-              checkAbortedTimeThreshold && rs.getLong(4) + abortedTimeThreshold < systemTime;
-          int numAbortedTxns = rs.getInt(5);
-          if (numAbortedTxns > abortedThreshold || pastTimeThreshold) {
-            CompactionInfo info = new CompactionInfo();
-            info.dbname = rs.getString(1);
-            info.tableName = rs.getString(2);
-            info.partName = rs.getString(3);
-            info.tooManyAborts = numAbortedTxns > abortedThreshold;
-            info.hasOldAbort = pastTimeThreshold;
-            LOG.debug("Found potential compaction: {}", info);
-            response.add(info);
+        if (!MetastoreConf.getBoolVar(conf, ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER)) {

Review Comment:
   The plan is to keep this config till some point until we know that abort cleanup is stable. If there are any issues with the given handler, we can use the compaction cycle. I will create a task once its determined that this feature is stable and we can remove this feature flag then.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;

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] sonarcloud[bot] commented on pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {

Review Comment:
   Modified `createValidTxnListForCleaner` to handle both aborts in one single method. Added necessary Javadoc. 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] veghlaci05 commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AcidTxnCleaner.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.ValidReadTxnList;
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleanupRequest.CleanupRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.commons.collections.ListUtils.subtract;
+
+/**
+ * An abstract class extending TaskHandler which contains the common methods from
+ * CompactionCleaner and AbortedTxnCleaner.
+ */
+abstract class AcidTxnCleaner extends TaskHandler {

Review Comment:
   Why not merge this class with `TaskHandler`? Both are abstract, and TaskHandler has no other subclasses. I see no reason keeping these classes separate.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {

Review Comment:
   +1 I think we should have a dedicated Unit test class for 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 #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [14 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java:
##########
@@ -649,6 +649,10 @@ public enum ConfVars {
     COMPACTOR_CLEANER_TABLECACHE_ON("metastore.compactor.cleaner.tablecache.on",
             "hive.compactor.cleaner.tablecache.on", true,
             "Enable table caching in the cleaner. Currently the cache is cleaned after each cycle."),
+    COMPACTOR_CLEAN_ABORTS_USING_CLEANER("metastore.compactor.clean.aborts.using.cleaner", "hive.compactor.clean.aborts.using.cleaner", true,

Review Comment:
   no need for extra config



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?
+            Long.MAX_VALUE : validWriteIdList.getMinOpenWriteId() - 1, validWriteIdList.getHighWatermark());
+    Table table = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
+
+    boolean success = cleanAndVerifyObsoleteDirectories(info, location, validWriteIdList, table);
+    if (success || CompactorUtil.isDynPartAbort(table, info.partName)) {
+      txnHandler.markCleanedForAborts(info);

Review Comment:
   same as above, could we make it generic and based on type do what's 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] deniskuzZ commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?

Review Comment:
   why would validWriteIdList.getMinOpenWriteId() be null?
   should we extract this logic in `getValidCleanerWriteIdList` method?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java:
##########
@@ -337,18 +287,9 @@ private static String idWatermark(CompactionInfo ci) {
     return " id=" + ci.id;
   }
 
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)
+  private ValidReaderWriteIdList getValidCleanerWriteIdListForCompactionCleaner(CompactionInfo ci, ValidTxnList validTxnList)

Review Comment:
   why rename here, just override the parent method and call super?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   Until and unless metadata of the aborted transactions is not explicitly removed by AbortedTxnCleaner, the AbortedTxnCleaner always fetches aborted entries in TXN_COMPONENTS table. So there is an implicit functionality of a retry operation already in case of failures during abort cleanup.



-- 
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 #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [11 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -84,6 +84,28 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     // string as they'd have to know which object to instantiate
     return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
   }
+
+  public static ValidTxnList createValidTxnListForTxnAbortedCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWaterMark = minOpenTxn - 1;

Review Comment:
   Changed. 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] SourabhBadhya commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java:
##########
@@ -1064,7 +1068,80 @@ public void testCleanAbortCompactAfterAbort() throws Exception {
     connection2.close();
   }
 
+  @Test
+  public void testAbortAfterMarkCleaned() throws Exception {
+    boolean useCleanerForAbortCleanup = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER);

Review Comment:
   Nice approach. Implemented it. 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] sonarcloud[bot] commented on pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [12 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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] veghlaci05 commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiatorWithAbortCleanupUsingCleaner.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+
+public class TestInitiatorWithAbortCleanupUsingCleaner extends TestInitiator {
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, true);

Review Comment:
   It should be done the other way around: you should set it false here and rename the class to TestInitiatorWithAbortCleanupUsingHandler or sth similar. In the base class it will take the default value which is true. I suggest doing the same in the other class hierarchy as well (CompactorTest and descendants). For me, it is more straightforward that a descendant class where the only change is a config value should test the non-default value, rather than restore the default value which was set to sth else in the base class.



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java:
##########
@@ -81,4 +102,63 @@ protected Partition resolvePartition(String dbName, String tableName, String par
       return null;
     }
   }
+
+  protected ValidReaderWriteIdList getValidCleanerWriteIdList(AcidTxnInfo acidTxnInfo, ValidTxnList validTxnList)

Review Comment:
   should we rename this method to getValidWriteIdList?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java:
##########
@@ -702,6 +699,102 @@ public void markCleaned(CompactionInfo info) throws MetaException {
     }
   }
 
+  @Override
+  public void markCleanedForAborts(AcidTxnInfo info) throws MetaException {
+    // Do cleanup of TXN_COMPONENTS table
+    LOG.debug("Running markCleanedForAborts with CompactionInfo: {}", info);
+    try {
+      Connection dbConn = null;
+      try {
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction);
+        markAbortCleaned(dbConn, info);
+        LOG.debug("Going to commit");
+        dbConn.commit();
+      } catch (SQLException e) {
+        LOG.error("Unable to delete from txn components due to {}", e.getMessage());
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        checkRetryable(e, "markCleanedForAborts(" + info + ")");
+        throw new MetaException("Unable to connect to transaction database " +
+                e.getMessage());
+      } finally {
+        closeDbConn(dbConn);
+      }
+    } catch (RetryException e) {
+      markCleanedForAborts(info);
+    }
+  }
+
+  private void markAbortCleaned(Connection dbConn, AcidTxnInfo info) throws MetaException, RetryException {

Review Comment:
   rename to `removeTxnComponents`



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?
+            Long.MAX_VALUE : validWriteIdList.getMinOpenWriteId() - 1, validWriteIdList.getHighWatermark());
+    Table table = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
+
+    boolean success = cleanAndVerifyObsoleteDirectories(info, location, validWriteIdList, table);
+    if (success || CompactorUtil.isDynPartAbort(table, info.partName)) {

Review Comment:
   This is part of the cleanup task when the config will be removed. Until then this is supposed to be here.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?
+            Long.MAX_VALUE : validWriteIdList.getMinOpenWriteId() - 1, validWriteIdList.getHighWatermark());
+    Table table = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
+
+    boolean success = cleanAndVerifyObsoleteDirectories(info, location, validWriteIdList, table);
+    if (success || CompactorUtil.isDynPartAbort(table, info.partName)) {
+      txnHandler.markCleanedForAborts(info);

Review Comment:
   Used `markCleaned(info, isAbortOnly)` for making it generic.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java:
##########
@@ -337,18 +287,9 @@ private static String idWatermark(CompactionInfo ci) {
     return " id=" + ci.id;
   }
 
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)
+  private ValidReaderWriteIdList getValidCleanerWriteIdListForCompactionCleaner(CompactionInfo ci, ValidTxnList validTxnList)

Review Comment:
   Implemented it, 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorWithAbortCleanupUsingCompactionCycle.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCompactor;
+import org.junit.Before;
+
+public class TestCompactorWithAbortCleanupUsingCompactionCycle extends TestCompactor {

Review Comment:
   ok, do we have a follow-up jira for 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] veghlaci05 merged pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {

Review Comment:
   that shouldn't be required for aborts



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java:
##########
@@ -516,6 +516,19 @@ Set<CompactionInfo> findPotentialCompactions(int abortedThreshold, long abortedT
   @RetrySemantics.ReadOnly
   List<CompactionInfo> findReadyToClean(long minOpenTxnWaterMark, long retentionTime) throws MetaException;
 
+  /**
+   * Find the aborted entries in TXN_COMPONENTS which can be used to
+   * clean directories belonging to transactions in aborted state.
+   * @param abortedTimeThreshold Age of table/partition's oldest aborted transaction involving a given table
+   *                            or partition that will trigger cleanup.
+   * @param abortedThreshold Number of aborted transactions involving a given table or partition
+   *                         that will trigger cleanup.
+   * @return Information of potential abort items that needs to be cleaned.
+   * @throws MetaException
+   */
+  @RetrySemantics.ReadOnly
+  List<AcidTxnInfo> findReadyToCleanForAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException;

Review Comment:
   I wouldn't introduce new API for 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] deniskuzZ commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java:
##########
@@ -541,6 +554,15 @@ Set<CompactionInfo> findPotentialCompactions(int abortedThreshold, long abortedT
   @RetrySemantics.CannotRetry
   void markCleaned(CompactionInfo info) throws MetaException;
 
+  /**
+   * This will remove an aborted entries from TXN_COMPONENTS table after
+   * the aborted directories are removed from the filesystem.
+   * @param info info on the aborted directories cleanup that needs to be removed
+   * @throws MetaException
+   */
+  @RetrySemantics.CannotRetry
+  void markCleanedForAborts(AcidTxnInfo info) throws MetaException;

Review Comment:
   I wouldn't create a separate API just for 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] SourabhBadhya commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {

Review Comment:
   Added dedicated unit test class. 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] sonarcloud[bot] commented on pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [22 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?
+            Long.MAX_VALUE : validWriteIdList.getMinOpenWriteId() - 1, validWriteIdList.getHighWatermark());
+    Table table = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
+
+    boolean success = cleanAndVerifyObsoleteDirectories(info, location, validWriteIdList, table);
+    if (success || CompactorUtil.isDynPartAbort(table, info.partName)) {

Review Comment:
   👍 



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
+    long[] exceptions = new long[txns.getOpen_txnsSize()];
+    int i = 0;
+    BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
+    // getOpen_txns() guarantees that the list contains only aborted & open txns.
+    // exceptions list must contain both txn types since validWriteIdList filters out the aborted ones and valid ones for that table.
+    // If a txn is not in exception list, it is considered as a valid one and thought of as an uncompacted write.
+    // See TxnHandler#getValidWriteIdsForTable() for more details.
+    for(long txnId : txns.getOpen_txns()) {

Review Comment:
   This loop is limited by the value of highWatermark. Mainly used for creating the exception list.



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
+    long[] exceptions = new long[txns.getOpen_txnsSize()];
+    int i = 0;
+    BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
+    // getOpen_txns() guarantees that the list contains only aborted & open txns.
+    // exceptions list must contain both txn types since validWriteIdList filters out the aborted ones and valid ones for that table.
+    // If a txn is not in exception list, it is considered as a valid one and thought of as an uncompacted write.
+    // See TxnHandler#getValidWriteIdsForTable() for more details.
+    for(long txnId : txns.getOpen_txns()) {
+      if(txnId > highWatermark) {
+        break;
+      }
+      exceptions[i] = txnId;
+      i++;
+    }
+    exceptions = Arrays.copyOf(exceptions, i);
+    //add ValidCleanerTxnList? - could be problematic for all the places that read it from

Review Comment:
   Removed it. Done.



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {

Review Comment:
   I have renamed `createValidTxnListForCleaner` to `createValidTxnListForCompactionCleaner`. This is different from `createValidTxnListForAbortedTxnCleaner`, mainly that we dont truncate the abortedBits which seems unnecessary. We are also not concerned if there are open txns from other tables present in this list (open txn on the same table will obviously be handled since highWatermark will be updated to min open for that table - 1). We just create an exception list based on the highWatermark and use it for the creating the validWriteIdList.



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [12 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [12 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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] akshat0395 commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   Thanks @SourabhBadhya, this works.



-- 
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 #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3273,11 +3273,11 @@ public static enum ConfVars {
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
-        "a major compaction."),
+        "a major compaction / cleanup of aborted directories."),
 
     HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD("hive.compactor.aborted.txn.time.threshold", "12h",
         new TimeValidator(TimeUnit.HOURS),
-        "Age of table/partition's oldest aborted transaction when compaction will be triggered. " +
+        "Age of table/partition's oldest aborted transaction when compaction / cleanup of aborted directories will be triggered. " +

Review Comment:
   same as above



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java:
##########
@@ -259,49 +247,11 @@ private void cleanUsingAcidDir(CompactionInfo ci, String location, long minOpenT
      */
 
     // Creating 'reader' list since we are interested in the set of 'obsolete' files
-    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(ci, validTxnList);
-    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
-
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-
-    // Collect all the files/dirs
-    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, path);
-    AcidDirectory dir = AcidUtils.getAcidState(fs, path, conf, validWriteIdList, Ref.from(false), false,
-            dirSnapshots);
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdListForCompactionCleaner(ci, validTxnList);
     Table table = metadataCache.computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.tableName));
-    boolean isDynPartAbort = CompactorUtil.isDynPartAbort(table, ci.partName);
-
-    List<Path> obsoleteDirs = CompactorUtil.getObsoleteDirs(dir, isDynPartAbort);
-    if (isDynPartAbort || dir.hasUncompactedAborts()) {
-      ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
-    }
-
-    List<Path> deleted = fsRemover.clean(new CleanupRequestBuilder().setLocation(location)
-            .setDbName(ci.dbname).setFullPartitionName(ci.getFullPartitionName())
-            .setRunAs(ci.runAs).setObsoleteDirs(obsoleteDirs).setPurge(true)
-            .build());
-
-    if (!deleted.isEmpty()) {
-      AcidMetricService.updateMetricsFromCleaner(ci.dbname, ci.tableName, ci.partName, dir.getObsolete(), conf,
-              txnHandler);
-    }
-
-    // Make sure there are no leftovers below the compacted watermark
-    boolean success = false;
-    conf.set(ValidTxnList.VALID_TXNS_KEY, new ValidReadTxnList().toString());
-    dir = AcidUtils.getAcidState(fs, path, conf, new ValidReaderWriteIdList(
-                    ci.getFullTableName(), new long[0], new BitSet(), ci.highestWriteId, Long.MAX_VALUE),
-            Ref.from(false), false, dirSnapshots);
+    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
 
-    List<Path> remained = subtract(CompactorUtil.getObsoleteDirs(dir, isDynPartAbort), deleted);
-    if (!remained.isEmpty()) {
-      LOG.warn("{} Remained {} obsolete directories from {}. {}",
-              idWatermark(ci), remained.size(), location, CompactorUtil.getDebugInfo(remained));
-    } else {
-      LOG.debug("{} All cleared below the watermark: {} from {}", idWatermark(ci), ci.highestWriteId, location);
-      success = true;
-    }
+    boolean success = cleanAndVerifyObsoleteDirectories(ci, location, validWriteIdList, table);

Review Comment:
   1 line below no need to check for `isDynPartAbort `



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
+    long[] exceptions = new long[txns.getOpen_txnsSize()];
+    int i = 0;
+    BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
+    // getOpen_txns() guarantees that the list contains only aborted & open txns.
+    // exceptions list must contain both txn types since validWriteIdList filters out the aborted ones and valid ones for that table.
+    // If a txn is not in exception list, it is considered as a valid one and thought of as an uncompacted write.
+    // See TxnHandler#getValidWriteIdsForTable() for more details.
+    for(long txnId : txns.getOpen_txns()) {

Review Comment:
   txns.getOpen_txns() is sorted so no need for whole list scan



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -61,12 +60,10 @@ public void init(AtomicBoolean stop) throws Exception {
     cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
             conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_THREADS_NUM),
             COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
-    if (CollectionUtils.isEmpty(cleanupHandlers)) {
-      FSRemover fsRemover = new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache);
-      cleanupHandlers = TaskHandlerFactory.getInstance()
-              .getHandlers(conf, txnHandler, metadataCache,
-                      metricsEnabled, fsRemover);
-    }
+    FSRemover fsRemover = new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache);
+    cleanupHandlers = TaskHandlerFactory.getInstance()
+            .getHandlers(conf, txnHandler, metadataCache,
+                    metricsEnabled, fsRemover);

Review Comment:
   could we move this to above line



-- 
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] akshat0395 commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java:
##########
@@ -76,10 +83,7 @@
 import static org.apache.hadoop.hive.ql.TestTxnCommands2.runWorker;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.*;

Review Comment:
   I see Mockito is already imported on line 67, we should avoid wildcard imports and let import specific classes that are required



##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java:
##########
@@ -1064,7 +1068,80 @@ public void testCleanAbortCompactAfterAbort() throws Exception {
     connection2.close();
   }
 
+  @Test
+  public void testAbortAfterMarkCleaned() throws Exception {
+    boolean useCleanerForAbortCleanup = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER);

Review Comment:
   We can avoid using this config all together for this test as the only place I can see this being use is for the `if` statement.
   As the entire test logic depend on COMPACTOR_CLEAN_ABORTS_USING_CLEANER being true, we can simply use 
   assumeTrue method and run the test case. `assumeTrue(MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER)` to run the test case 
   The default value of this config is also true, Hence I feel this if condition can be avoided. WDYT
   



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {

Review Comment:
   Do we have unit test for this class?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   Should we add a retry mechanism? Any thoughts @SourabhBadhya @veghlaci05 



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {

Review Comment:
   We should also capture more specific exception related to location resolution, this will help identifying access/location related issues faster



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -84,6 +84,28 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     // string as they'd have to know which object to instantiate
     return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
   }
+
+  public static ValidTxnList createValidTxnListForTxnAbortedCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWaterMark = minOpenTxn - 1;

Review Comment:
   nit: Watermark is a single word, we can rename to highWatermark



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java:
##########
@@ -702,6 +699,102 @@ public void markCleaned(CompactionInfo info) throws MetaException {
     }
   }
 
+  @Override
+  public void markCleanedForAborts(AcidTxnInfo info) throws MetaException {
+    // Do cleanup of TXN_COMPONENTS table
+    LOG.debug("Running markCleanedForAborts with CompactionInfo: {}", info);
+    try {
+      Connection dbConn = null;
+      try {
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction);
+        markAbortCleaned(dbConn, info);
+        LOG.debug("Going to commit");
+        dbConn.commit();
+      } catch (SQLException e) {
+        LOG.error("Unable to delete from txn components due to {}", e.getMessage());
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        checkRetryable(e, "markCleanedForAborts(" + info + ")");
+        throw new MetaException("Unable to connect to transaction database " +
+                e.getMessage());
+      } finally {
+        closeDbConn(dbConn);
+      }
+    } catch (RetryException e) {
+      markCleanedForAborts(info);
+    }
+  }
+
+  private void markAbortCleaned(Connection dbConn, AcidTxnInfo info) throws MetaException, RetryException {

Review Comment:
   Renamed. Done.



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java:
##########
@@ -516,6 +516,19 @@ Set<CompactionInfo> findPotentialCompactions(int abortedThreshold, long abortedT
   @RetrySemantics.ReadOnly
   List<CompactionInfo> findReadyToClean(long minOpenTxnWaterMark, long retentionTime) throws MetaException;
 
+  /**
+   * Find the aborted entries in TXN_COMPONENTS which can be used to
+   * clean directories belonging to transactions in aborted state.
+   * @param abortedTimeThreshold Age of table/partition's oldest aborted transaction involving a given table
+   *                            or partition that will trigger cleanup.
+   * @param abortedThreshold Number of aborted transactions involving a given table or partition
+   *                         that will trigger cleanup.
+   * @return Information of potential abort items that needs to be cleaned.
+   * @throws MetaException
+   */
+  @RetrySemantics.ReadOnly
+  List<AcidTxnInfo> findReadyToCleanForAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException;

Review Comment:
   Renamed. Done.



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java:
##########
@@ -541,6 +554,15 @@ Set<CompactionInfo> findPotentialCompactions(int abortedThreshold, long abortedT
   @RetrySemantics.CannotRetry
   void markCleaned(CompactionInfo info) throws MetaException;
 
+  /**
+   * This will remove an aborted entries from TXN_COMPONENTS table after
+   * the aborted directories are removed from the filesystem.
+   * @param info info on the aborted directories cleanup that needs to be removed
+   * @throws MetaException
+   */
+  @RetrySemantics.CannotRetry
+  void markCleanedForAborts(AcidTxnInfo info) throws MetaException;

Review Comment:
   Removed the API. Extended `markCleaned` with a boolean `isAbortOnly` flag. 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3273,11 +3273,11 @@ public static enum ConfVars {
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
-        "a major compaction."),
+        "a major compaction / cleanup of aborted directories."),

Review Comment:
   we are not doing compaction for aborts, just cleanup



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   can we place selected entries in compaction_queue in "r" state? if there would be some issue on s3, HMS might just die 
   



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   @deniskuzZ. I have set the config to `false` and will be addressing retry backoff in a separate PR.



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   @deniskuzZ Yes there is no retry backoff as of now. Even if we want to implement this, the information related to retry needs to be stored somewhere. Since we are not using COMPACTION_QUEUE, the only place I see this going is TXN_COMPONENTS table which I am not sure is desirable or not.



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {

Review Comment:
   min open txn would be still capped by highWatermark in ValidReadTxnList constructor, result should be still the same



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {

Review Comment:
   Removed. Done.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {

Review Comment:
   Removed. Done.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);

Review Comment:
   Extended `markCleaned` API with isAbortOnly boolean flag which does abort cleanup when the flag is true. Done.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?

Review Comment:
   `validWriteIdList.getMinOpenWriteId()` returns NULL when its internal variable `minOpenWriteId` is Long.MAX_VALUE. There are usages in other parts of code wherein it is expected to be NULL when the internal variable `minOpenWriteId` is Long.MAX_VALUE.



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java:
##########
@@ -76,10 +83,7 @@
 import static org.apache.hadoop.hive.ql.TestTxnCommands2.runWorker;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.*;

Review Comment:
   Removed wildcard imports and added specific imports. 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3273,11 +3273,11 @@ public static enum ConfVars {
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
-        "a major compaction."),
+        "a major compaction / cleanup of aborted directories."),

Review Comment:
   would it actually trigger compaction?
   PS: we should deprecate. this config on HS2 side



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {

Review Comment:
   that shouldn't be required for aborts



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   Initiator had backoff based on retry count and time. Right now, do we have any back-off? If not, can we reuse Initiator logic 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] deniskuzZ commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorWithAbortCleanupUsingCompactionCycle.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCompactor;
+import org.junit.Before;
+
+public class TestCompactorWithAbortCleanupUsingCompactionCycle extends TestCompactor {

Review Comment:
   I don't think that should be supported anymore



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);

Review Comment:
   can we do proper handling in markCleaned based on a req type (isAbort)?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java:
##########
@@ -337,18 +287,9 @@ private static String idWatermark(CompactionInfo ci) {
     return " id=" + ci.id;
   }
 
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)
+  private ValidReaderWriteIdList getValidCleanerWriteIdListForCompactionCleaner(CompactionInfo ci, ValidTxnList validTxnList)

Review Comment:
   why rename 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] deniskuzZ commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3273,11 +3273,11 @@ public static enum ConfVars {
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
-        "a major compaction."),
+        "a major compaction / cleanup of aborted directories."),

Review Comment:
   would it actually trigger compaction?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
+    long[] exceptions = new long[txns.getOpen_txnsSize()];
+    int i = 0;
+    BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
+    // getOpen_txns() guarantees that the list contains only aborted & open txns.
+    // exceptions list must contain both txn types since validWriteIdList filters out the aborted ones and valid ones for that table.
+    // If a txn is not in exception list, it is considered as a valid one and thought of as an uncompacted write.
+    // See TxnHandler#getValidWriteIdsForTable() for more details.
+    for(long txnId : txns.getOpen_txns()) {

Review Comment:
   reformat, missing space



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -60,20 +60,20 @@
 public class TxnUtils {
   private static final Logger LOG = LoggerFactory.getLogger(TxnUtils.class);
 
-  public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns, long minOpenTxnGLB) {
-    long highWaterMark = minOpenTxnGLB - 1;
+  public static ValidTxnList createValidTxnListForCompactionCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
     long[] abortedTxns = new long[txns.getOpen_txnsSize()];
     BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
     int i = 0;
     for(long txnId : txns.getOpen_txns()) {
-      if(txnId > highWaterMark) {
+      if(txnId > highWatermark) {

Review Comment:
   space



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiatorWithAbortCleanupUsingCleaner.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+
+public class TestInitiatorWithAbortCleanupUsingCleaner extends TestInitiator {
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, true);

Review Comment:
   Implemented it. Done.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TxnAbortedCleaner.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+class TxnAbortedCleaner extends AcidTxnCleaner {

Review Comment:
   Renamed. 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] veghlaci05 commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   I think it does not worth doing it
   
   1. It's a metrics, not a critical component or code part
   2. adding a retry (especially with back-off would possibly corrupt the metrics value, as the perfLogEnd call would not be in sync with the complete time of the operation it measures



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   @deniskuzZ Makes sense. I have set the config to `false` and will be addressing retry backoff in a separate PR.



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {

Review Comment:
   I have renamed `createValidTxnListForCleaner` to `createValidTxnListForCompactionCleaner`. This is different from `createValidTxnListForAbortedTxnCleaner`, mainly that we dont truncate the abortedBits which seems unnecessary. We are also not concerned if there are open txns from other tables present in this list (open txn on the same table will obviously be handled since highWatermark will be updated to min open txn for that table - 1). We just create an exception list based on the highWatermark and use it for the creating the validWriteIdList.



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {

Review Comment:
   As discussed, this is already handled here - 
   The job of cleanup is run only when the table access is available either to the user who is running the cleanup or table owner. If none is having the permission or if the location itself is missing, then appropriate error is logged and the error is propogated as well.
   The relevant method is - `TxnUtils#findUserToRunAs`



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?
+            Long.MAX_VALUE : validWriteIdList.getMinOpenWriteId() - 1, validWriteIdList.getHighWatermark());
+    Table table = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+    LOG.debug("Cleaning based on writeIdList: {}", validWriteIdList);
+
+    boolean success = cleanAndVerifyObsoleteDirectories(info, location, validWriteIdList, table);
+    if (success || CompactorUtil.isDynPartAbort(table, info.partName)) {

Review Comment:
   isDynPartAbort could be removed from CompactionCleaner as it doesn't handle aborts anymore



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends TaskHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void abortCleanUsingAcidDir(AcidTxnInfo info, String location, long minOpenTxn) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForAbortedTxnCleaner(txnHandler.getOpenTxns(), minOpenTxn);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+
+    ValidReaderWriteIdList validWriteIdList = getValidCleanerWriteIdList(info, validTxnList);
+
+    // Set the highestWriteId of the cleanup equal to the min(minOpenWriteId - 1, highWatermark).
+    // This is necessary for looking at the complete state of the table till the min open write Id
+    // (if there is an open txn on the table) or the highestWatermark.
+    // This is used later on while deleting the records in TXN_COMPONENTS table.
+    info.highestWriteId = Math.min(isNull(validWriteIdList.getMinOpenWriteId()) ?

Review Comment:
   why would validWriteIdList.getMinOpenWriteId() be null?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
+    long[] exceptions = new long[txns.getOpen_txnsSize()];
+    int i = 0;
+    BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
+    // getOpen_txns() guarantees that the list contains only aborted & open txns.
+    // exceptions list must contain both txn types since validWriteIdList filters out the aborted ones and valid ones for that table.
+    // If a txn is not in exception list, it is considered as a valid one and thought of as an uncompacted write.
+    // See TxnHandler#getValidWriteIdsForTable() for more details.
+    for(long txnId : txns.getOpen_txns()) {
+      if(txnId > highWatermark) {
+        break;
+      }
+      exceptions[i] = txnId;
+      i++;
+    }
+    exceptions = Arrays.copyOf(exceptions, i);
+    //add ValidCleanerTxnList? - could be problematic for all the places that read it from

Review Comment:
   is this a leftover comment?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   oh, we do, it's time based: handleCleanerAttemptFailure



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorWithAbortCleanupUsingCompactionCycle.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCompactor;
+import org.junit.Before;
+
+public class TestCompactorWithAbortCleanupUsingCompactionCycle extends TestCompactor {

Review Comment:
   i don't think that should be supported any more



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AcidTxnCleaner.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.ValidReadTxnList;
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleanupRequest.CleanupRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.commons.collections.ListUtils.subtract;
+
+/**
+ * An abstract class extending TaskHandler which contains the common methods from
+ * CompactionCleaner and AbortedTxnCleaner.
+ */
+abstract class AcidTxnCleaner extends TaskHandler {

Review Comment:
   Removed AcidTxnCleaner and pushed all methods to TaskHandler. 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] SourabhBadhya commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   @deniskuzZ Yes there is no retry backoff as of now. Even if we want to implement this needs to be stored somewhere. Since we are not using COMPACTION_QUEUE, the only place I see this going is TXN_COMPONENTS table which I am not sure is desirable or not.



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -60,20 +60,20 @@
 public class TxnUtils {
   private static final Logger LOG = LoggerFactory.getLogger(TxnUtils.class);
 
-  public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns, long minOpenTxnGLB) {
-    long highWaterMark = minOpenTxnGLB - 1;
+  public static ValidTxnList createValidTxnListForCompactionCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
     long[] abortedTxns = new long[txns.getOpen_txnsSize()];
     BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
     int i = 0;
     for(long txnId : txns.getOpen_txns()) {
-      if(txnId > highWaterMark) {
+      if(txnId > highWatermark) {

Review Comment:
   Done.



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -60,20 +60,20 @@
 public class TxnUtils {
   private static final Logger LOG = LoggerFactory.getLogger(TxnUtils.class);
 
-  public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns, long minOpenTxnGLB) {
-    long highWaterMark = minOpenTxnGLB - 1;
+  public static ValidTxnList createValidTxnListForCompactionCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
     long[] abortedTxns = new long[txns.getOpen_txnsSize()];
     BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
     int i = 0;
     for(long txnId : txns.getOpen_txns()) {
-      if(txnId > highWaterMark) {
+      if(txnId > highWatermark) {
         break;
       }
       if(abortedBits.get(i)) {

Review Comment:
   Done



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {
+    long highWatermark = minOpenTxn - 1;
+    long[] exceptions = new long[txns.getOpen_txnsSize()];
+    int i = 0;
+    BitSet abortedBits = BitSet.valueOf(txns.getAbortedBits());
+    // getOpen_txns() guarantees that the list contains only aborted & open txns.
+    // exceptions list must contain both txn types since validWriteIdList filters out the aborted ones and valid ones for that table.
+    // If a txn is not in exception list, it is considered as a valid one and thought of as an uncompacted write.
+    // See TxnHandler#getValidWriteIdsForTable() for more details.
+    for(long txnId : txns.getOpen_txns()) {

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] sonarcloud[bot] commented on pull request #4091: [DRAFT] HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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] veghlaci05 commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TxnAbortedCleaner.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+class TxnAbortedCleaner extends AcidTxnCleaner {

Review Comment:
   I think AbortedTxnCleaner is more meaningful



##########
ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiatorWithAbortCleanupUsingCleaner.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+
+public class TestInitiatorWithAbortCleanupUsingCleaner extends TestInitiator {
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, true);

Review Comment:
   This class is not a subclass of CompactorTest. It should be done the other way around: you should set it false here and rename the class to TestInitiatorWithAbortCleanupUsingHandler or sth similar. In the base class it will take the default value which is true. I suggest doing the same in the other class hierarchy as well (CompactorTest and descendants). For me, it is more straightforward that a descendant class where the only change is a config value should test the non-default value, rather than restore the default value which was set to sth else in the base class.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TxnAbortedCleaner.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+class TxnAbortedCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TxnAbortedCleaner.class.getName());
+
+  public TxnAbortedCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<CompactionInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) throws MetaException {

Review Comment:
   This class should have its own DAO/POJO class, we are starting to really overuse CompactionInfo



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   I think it does not worth doing it
   
   1. It's a metrics, not a critical component or code part
   2. adding a retry (especially with back-off) would possibly corrupt the metrics value, as the perfLogEnd call would not be in sync with the complete time of the operation it measures



-- 
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] akshat0395 commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   @veghlaci05 My bad, when I mentioned retry I commented on the finally block, what I meant to do is trigger a conversation around retry mechanism for the abortCleanUsingAcidDir not the metrics



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidTxnInfo.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.hadoop.hive.metastore.txn;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
+import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+
+import java.util.Set;
+
+/**
+ * A class used for encapsulating information of abort-cleanup activities and compaction activities.
+ */
+public class AcidTxnInfo {

Review Comment:
   Can we reuse CompactionInfo object and not create another entity?



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java:
##########
@@ -82,7 +82,29 @@ public static ValidTxnList createValidTxnListForCleaner(GetOpenTxnsResponse txns
     bitSet.set(0, abortedTxns.length);
     //add ValidCleanerTxnList? - could be problematic for all the places that read it from
     // string as they'd have to know which object to instantiate
-    return new ValidReadTxnList(abortedTxns, bitSet, highWaterMark, Long.MAX_VALUE);
+    return new ValidReadTxnList(abortedTxns, bitSet, highWatermark, Long.MAX_VALUE);
+  }
+
+  public static ValidTxnList createValidTxnListForAbortedTxnCleaner(GetOpenTxnsResponse txns, long minOpenTxn) {

Review Comment:
   how is that different from `createValidTxnListForCleaner `, everything in Open_txns list `< minOpenTxn - 1` would be aborted



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java:
##########
@@ -162,31 +162,33 @@ public Set<CompactionInfo> findPotentialCompactions(int abortedThreshold,
         }
         rs.close();
 
-        // Check for aborted txns: number of aborted txns past threshold and age of aborted txns
-        // past time threshold
-        boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0;
-        String sCheckAborted = "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " +
-          "MIN(\"TXN_STARTED\"), COUNT(*) FROM \"TXNS\", \"TXN_COMPONENTS\" " +
-          "   WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + " " +
-          "GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " +
-              (checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold);
-
-        LOG.debug("Going to execute query <{}>", sCheckAborted);
-        rs = stmt.executeQuery(sCheckAborted);
-        long systemTime = System.currentTimeMillis();
-        while (rs.next()) {
-          boolean pastTimeThreshold =
-              checkAbortedTimeThreshold && rs.getLong(4) + abortedTimeThreshold < systemTime;
-          int numAbortedTxns = rs.getInt(5);
-          if (numAbortedTxns > abortedThreshold || pastTimeThreshold) {
-            CompactionInfo info = new CompactionInfo();
-            info.dbname = rs.getString(1);
-            info.tableName = rs.getString(2);
-            info.partName = rs.getString(3);
-            info.tooManyAborts = numAbortedTxns > abortedThreshold;
-            info.hasOldAbort = pastTimeThreshold;
-            LOG.debug("Found potential compaction: {}", info);
-            response.add(info);
+        if (!MetastoreConf.getBoolVar(conf, ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER)) {

Review Comment:
   no need for that, leads to code duplication



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java:
##########
@@ -464,6 +466,54 @@ public List<CompactionInfo> findReadyToClean(long minOpenTxnWaterMark, long rete
     }
   }
 
+  @Override
+  @RetrySemantics.ReadOnly
+  public List<AcidTxnInfo> findReadyToCleanForAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException {

Review Comment:
   rename `findReadyToCleanAborts`



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   Initiator had backoff based on retry count and time. Right now do we have any back-off? If no, can we reuse Initiator logic 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] deniskuzZ commented on a diff in pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   also we had time based back-off in Cleaner (handleCleanerAttemptFailure) based on Compaction_Queue table, but that won't work for aborts now



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   Initiator had backoff based on retry count and time. Right now, do we have any back-off? 



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TxnAbortedCleaner.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+class TxnAbortedCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TxnAbortedCleaner.class.getName());
+
+  public TxnAbortedCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<CompactionInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) throws MetaException {

Review Comment:
   Added a class called AcidTxnInfo. CompactionInfo now extends this class. 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] sonarcloud[bot] commented on pull request #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [22 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info,
+              e.getMessage());
+      throw new MetaException(e.getMessage());
+    } finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(AbortedTxnCleaner.class.getName(), cleanerMetric);

Review Comment:
   @deniskuzZ Yes there is no retry backoff as of now. Even if we want to implement this, the information related to retry backoff needs to be stored somewhere. Since we are not using COMPACTION_QUEUE, the only place I see this going is TXN_COMPONENTS table which I am not sure is desirable or not.



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

To unsubscribe, e-mail: 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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3273,11 +3273,11 @@ public static enum ConfVars {
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
-        "a major compaction."),
+        "a major compaction / cleanup of aborted directories."),

Review Comment:
   This config is used for controlling the number of aborted txns after which the cleanup is triggered (either cleanup / compaction) depending on what is expected.



##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3273,11 +3273,11 @@ public static enum ConfVars {
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
-        "a major compaction."),
+        "a major compaction / cleanup of aborted directories."),
 
     HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD("hive.compactor.aborted.txn.time.threshold", "12h",
         new TimeValidator(TimeUnit.HOURS),
-        "Age of table/partition's oldest aborted transaction when compaction will be triggered. " +
+        "Age of table/partition's oldest aborted transaction when compaction / cleanup of aborted directories will be triggered. " +

Review Comment:
   This config is used for controlling the time threshold of aborted txns after which the cleanup is triggered (either cleanup / compaction) depending on what is expected.



##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorBase.java:
##########
@@ -89,6 +89,7 @@ public void setup() throws Exception {
     hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false);
     MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true);
     MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true);
+    MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, false);

Review Comment:
   The plan is to keep this config till some point until we know that abort cleanup is stable. If there are any issues with the given handler, we can use the compaction cycle. I will create a task once its determined that this feature is stable and we can remove this feature flag then.



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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

   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=4091)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4091&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=4091&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=4091&resolved=false&types=CODE_SMELL) [10 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4091&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=4091&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=4091&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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -3273,11 +3273,11 @@ public static enum ConfVars {
 
     HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000,
         "Number of aborted transactions involving a given table or partition that will trigger\n" +
-        "a major compaction."),
+        "a major compaction / cleanup of aborted directories."),

Review Comment:
   don't forget to update this in future



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorWithAbortCleanupUsingCompactionCycle.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor;
+
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCompactor;
+import org.junit.Before;
+
+public class TestCompactorWithAbortCleanupUsingCompactionCycle extends TestCompactor {

Review Comment:
   I created one just now - https://issues.apache.org/jira/browse/HIVE-27270



-- 
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 #4091: HIVE-27020: Implement a separate handler to handle aborted transaction cleanup

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.hadoop.hive.ql.txn.compactor.handler;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.txn.AcidTxnInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.isNull;
+
+/**
+ * Abort-cleanup based implementation of TaskHandler.
+ * Provides implementation of creation of abort clean tasks.
+ */
+class AbortedTxnCleaner extends AcidTxnCleaner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortedTxnCleaner.class.getName());
+
+  public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler,
+                           MetadataCache metadataCache, boolean metricsEnabled,
+                           FSRemover fsRemover) {
+    super(conf, txnHandler, metadataCache, metricsEnabled, fsRemover);
+  }
+
+  /**
+   The following cleanup is based on the following idea - <br>
+   1. Aborted cleanup is independent of compaction. This is because directories which are written by
+      aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which
+      only sees the aborted deltas and does not read the file).<br><br>
+
+   The following algorithm is used to clean the set of aborted directories - <br>
+      a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanForAborts(long, int)}).<br>
+      b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table. <br>
+      c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID <br>
+      d. Fetch the aborted directories and delete the directories. <br>
+      e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table.
+   **/
+  @Override
+  public List<Runnable> getTasks() throws MetaException {
+    int abortedThreshold = HiveConf.getIntVar(conf,
+              HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD);
+    long abortedTimeThreshold = HiveConf
+              .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
+                      TimeUnit.MILLISECONDS);
+    List<AcidTxnInfo> readyToCleanAborts = txnHandler.findReadyToCleanForAborts(abortedTimeThreshold, abortedThreshold);
+
+    if (!readyToCleanAborts.isEmpty()) {
+      return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() ->
+                      clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled)))
+              .collect(Collectors.toList());
+    }
+    return Collections.emptyList();
+  }
+
+  private void clean(AcidTxnInfo info, long minOpenTxn, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", info);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_";
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(AbortedTxnCleaner.class.getName(), cleanerMetric);
+      }
+      Table t;
+      Partition p = null;
+      t = metadataCache.computeIfAbsent(info.getFullTableName(), () -> resolveTable(info.dbname, info.tableName));
+      if (isNull(t)) {
+        // The table was dropped before we got around to cleaning it.
+        LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName());
+        txnHandler.markCleanedForAborts(info);
+        return;
+      }
+      if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
+        // The table was marked no clean up true.
+        LOG.info("Skipping table {} clean up, as NO_CLEANUP set to true", info.getFullTableName());
+        return;
+      }
+      if (!isNull(info.partName)) {
+        p = resolvePartition(info.dbname, info.tableName, info.partName);
+        if (isNull(p)) {
+          // The partition was dropped before we got around to cleaning it.
+          LOG.info("Unable to find partition {}, assuming it was dropped.",
+                  info.getFullPartitionName());
+          txnHandler.markCleanedForAborts(info);
+          return;
+        }
+        if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
+          // The partition was marked no clean up true.
+          LOG.info("Skipping partition {} clean up, as NO_CLEANUP set to true", info.getFullPartitionName());
+          return;
+        }
+      }
+
+      String location = CompactorUtil.resolveStorageDescriptor(t,p).getLocation();
+      info.runAs = TxnUtils.findUserToRunAs(location, t, conf);
+      abortCleanUsingAcidDir(info, location, minOpenTxn);
+
+    } catch (Exception e) {

Review Comment:
   As discussed, this is already handled here - `TxnUtils#findUserToRunAs`
   The job of cleanup is run only when the table access is available either to the user who is running the cleanup or table owner. If none is having the permission or if the location itself is missing, then appropriate error is logged and the error is propogated 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