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/02/07 07:53:57 UTC

[GitHub] [hive] SourabhBadhya opened a new pull request, #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   <!--
   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.
   -->
   Split Cleaner into separate manageable modular entities
   
   ### 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.
   -->
   This division allows for dynamic extensibility of cleanup from multiple handlers. Every handler is responsible for providing cleaning requests from a specific source.
   
   ### 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.
   -->
   Existing unit tests


-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionCleaningRequest.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A cleaning request class specific to compaction based cleanup.
+ */
+public class CompactionCleaningRequest extends CleaningRequest {
+
+  private final CompactionInfo compactionInfo;
+  private final Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots;
+
+  public CompactionCleaningRequest(String location, CompactionInfo info, List<Path> obsoleteDirs,
+                                   boolean purge, FileSystem fs, Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots,
+                                   boolean dropPartition) {
+    super(RequestType.COMPACTION, location, obsoleteDirs, purge, fs);
+    this.compactionInfo = info;
+    this.dbName = compactionInfo.dbname;
+    this.tableName = compactionInfo.tableName;
+    this.partitionName = compactionInfo.partName;
+    this.dirSnapshots = dirSnapshots;
+    this.cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
+            (compactionInfo.type != null ? compactionInfo.type.toString().toLowerCase() : null);
+    this.runAs = compactionInfo.runAs;
+    this.dropPartition = dropPartition;
+    this.fullPartitionName = compactionInfo.getFullPartitionName();
+  }
+
+  public CompactionInfo getCompactionInfo() {
+    return compactionInfo;
+  }
+
+  public Map<Path, AcidUtils.HdfsDirSnapshot> getHdfsDirSnapshots() {
+    return dirSnapshots;
+  }
+
+  @Override
+  public String toString() {

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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler<T extends CleaningRequest> {

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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

Review Comment:
   Cache is initialized based on a config - 
   In the current master this is code - 
   https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java#L114-L116
   
   I think accepting a boolean in constructor and initializing might also make sense. Return type is also not used anywhere. 



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java:
##########
@@ -329,9 +335,9 @@ public void init(AtomicBoolean stop) throws Exception {
     compactionExecutor = CompactorUtil.createExecutorWithThreadFactory(
             conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_REQUEST_QUEUE),
             COMPACTOR_INTIATOR_THREAD_NAME_FORMAT);
-    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,
-        MetastoreConf.ConfVars.COMPACTOR_INITIATOR_TABLECACHE_ON);
-    initializeCache(tableCacheOn);
+//    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,

Review Comment:
   These came in by-mistake. 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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CacheContainer.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class CacheContainer {

Review Comment:
   Done.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {

Review Comment:
   maybe `TaskHandler`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [24 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -194,8 +210,17 @@ private LockRequest createLockRequest(String dbName, String tableName, String pa
     }

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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionHandler.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+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.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CacheContainer;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest.CleaningRequestBuilder;
+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.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+import static java.util.Objects.isNull;
+
+/**
+ * A compaction based implementation of RequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionHandler extends RequestHandler {

Review Comment:
   Renamed it as `CompactionCleanHandler`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CacheContainer.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class CacheContainer {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   I think I will address this in a separate Jira.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   I don't think we need to wrap it with Optional in a current design, we could just check `!= 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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   @deniskuzZ Optional is required to check whether the cache is actually built or not.
   Related code in current master - https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java#L145-L154



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaningRequestHandler.java:
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest.CompactionCleaningRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+
+/**
+ * A compaction based implementation of CleaningRequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionCleaningRequestHandler extends CleaningRequestHandler<CompactionCleaningRequest> {

Review Comment:
   I have renamed as above but without CompactionRequest since that is not required after refactor.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;

Review Comment:
   Removed this from the request object.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final HiveConf conf;
+  private final ReplChangeManager replChangeManager;
+
+  public FSRemover(HiveConf conf, ReplChangeManager replChangeManager) {
+    this.conf = conf;
+    this.replChangeManager = replChangeManager;
+  }
+
+  public List<Path> clean(CleaningRequest cr) throws MetaException {
+    Ref<List<Path>> removedFiles = Ref.from(new ArrayList<>());
+    try {
+      Callable<List<Path>> cleanUpTask;
+      cleanUpTask = () -> removeFiles(cr);
+
+      if (CompactorUtil.runJobAsSelf(cr.runAs())) {
+        removedFiles.value = cleanUpTask.call();
+      } else {
+        LOG.info("Cleaning as user {} for {}", cr.runAs(), cr.getFullPartitionName());
+        UserGroupInformation ugi = UserGroupInformation.createProxyUser(cr.runAs(),
+                UserGroupInformation.getLoginUser());
+        try {
+          ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
+            removedFiles.value = cleanUpTask.call();
+            return null;
+          });
+        } finally {
+          try {
+            FileSystem.closeAllForUGI(ugi);
+          } catch (IOException exception) {
+            LOG.error("Could not clean up file-system handles for UGI: {} for {}",
+                    ugi, cr.getFullPartitionName(), exception);
+          }
+        }
+      }
+    } catch (Exception ex) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", cr,
+              StringUtils.stringifyException(ex));
+    }
+    return removedFiles.value;
+  }
+
+  /**
+   * @param cr Cleaning request
+   * @return List of deleted files if any files were removed
+   */
+  private List<Path> removeFiles(CleaningRequest cr)
+          throws MetaException, IOException {
+    List<Path> deleted = new ArrayList<>();
+    if (cr.getObsoleteDirs().isEmpty()) {
+      return deleted;
+    }
+    LOG.info("About to remove {} obsolete directories from {}. {}", cr.getObsoleteDirs().size(),
+            cr.getLocation(), CompactorUtil.getDebugInfo(cr.getObsoleteDirs()));
+    boolean needCmRecycle;
+    try {
+      Database db = getMSForConf(conf).getDatabase(getDefaultCatalog(conf), cr.getDbName());

Review Comment:
   if the metadata cache contains db as well why do we need to fetch it every time?



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

To unsubscribe, e-mail: 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaningRequestHandler.java:
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest.CompactionCleaningRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+
+/**
+ * A compaction based implementation of CleaningRequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionCleaningRequestHandler extends CleaningRequestHandler<CompactionCleaningRequest> {

Review Comment:
   maybe simply 
   ````
   CompactionCleaner extends FSCleaner<CleanupRequest>?
   ````



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestHandler.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.Cleaner;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCleaner;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+
+public class TestHandler extends TestCleaner {
+
+  @Test
+  public void testCompactionHandlerForSuccessfulCompaction() throws Exception {
+    Table t = newTable("default", "handler_success_table", true);
+    Partition p = newPartition(t, "today");
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
+
+    burnThroughTransactions(t.getDbName(), t.getTableName(), 25);
+
+    CompactionRequest rqst = new CompactionRequest(t.getDbName(), t.getTableName(), CompactionType.MAJOR);
+    rqst.setPartitionname("ds=today");
+    compactInTxn(rqst);
+
+    Handler handler = new CompactionHandler(conf, txnHandler, false);
+
+    // Fetch the compaction request using the handler
+    List<CleaningRequest> cleaningRequests = handler.findReadyToClean();
+    Assert.assertEquals(1, cleaningRequests.size());
+    CleaningRequest cr = cleaningRequests.get(0);
+    Assert.assertEquals(t.getDbName(), cr.getDbName());
+    Assert.assertEquals(t.getTableName(), cr.getTableName());
+    Assert.assertEquals("ds=today", cr.getPartitionName());
+    Assert.assertEquals(CleaningRequest.RequestType.COMPACTION, cr.getType());
+
+    // Check whether appropriate handler utility methods are called exactly once in a successful compaction scenario.
+    Handler mockedHandler = Mockito.spy(handler);
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Cleaner cleaner = new Cleaner(Arrays.asList(mockedHandler));
+    cleaner.setConf(conf);
+    cleaner.init(stop);
+    cleaner.run();
+
+    Mockito.verify(mockedHandler, Mockito.times(1)).findReadyToClean();
+    Mockito.verify(mockedHandler, Mockito.times(1)).beforeExecutingCleaningRequest(any(CleaningRequest.class));
+    Mockito.verify(mockedHandler, Mockito.times(1)).afterExecutingCleaningRequest(any(CleaningRequest.class), any(List.class));

Review Comment:
   Added assert. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -141,49 +93,37 @@ public void run() {
                     new CleanerCycleUpdater(MetricsConstants.COMPACTION_CLEANER_CYCLE_DURATION, startedAt));
           }
 
-          long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
-
-          checkInterrupt();
-
-          List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
-
-          checkInterrupt();
-
-          if (!readyToClean.isEmpty()) {
-            long minTxnIdSeenOpen = txnHandler.findMinTxnIdSeenOpen();
-            final long cleanerWaterMark =
-                minTxnIdSeenOpen < 0 ? minOpenTxnId : Math.min(minOpenTxnId, minTxnIdSeenOpen);
-
-            LOG.info("Cleaning based on min open txn id: " + cleanerWaterMark);
-            List<CompletableFuture<Void>> cleanerList = new ArrayList<>();
-            // For checking which compaction can be cleaned we can use the minOpenTxnId
-            // However findReadyToClean will return all records that were compacted with old version of HMS
-            // where the CQ_NEXT_TXN_ID is not set. For these compactions we need to provide minTxnIdSeenOpen
-            // to the clean method, to avoid cleaning up deltas needed for running queries
-            // when min_history_level is finally dropped, than every HMS will commit compaction the new way
-            // and minTxnIdSeenOpen can be removed and minOpenTxnId can be used instead.
-            for (CompactionInfo compactionInfo : readyToClean) {
-
-              //Check for interruption before scheduling each compactionInfo and return if necessary
+          for (Handler handler : handlers) {
+            try {
+              List<CleaningRequest> readyToClean = handler.findReadyToClean();
               checkInterrupt();
 
-              CompletableFuture<Void> asyncJob =
-                  CompletableFuture.runAsync(
-                          ThrowingRunnable.unchecked(() -> clean(compactionInfo, cleanerWaterMark, metricsEnabled)),
-                          cleanerExecutor)
-                      .exceptionally(t -> {
-                        LOG.error("Error clearing {}", compactionInfo.getFullPartitionName(), t);
-                        return null;
-                      });
-              cleanerList.add(asyncJob);
+              if (!readyToClean.isEmpty()) {
+                List<CompletableFuture<Void>> cleanerList = new ArrayList<>();
+                for (CleaningRequest cr : readyToClean) {
+
+                  //Check for interruption before scheduling each cleaning request and return if necessary
+                  checkInterrupt();
+
+                  CompletableFuture<Void> asyncJob = CompletableFuture.runAsync(
+                                  ThrowingRunnable.unchecked(new FSRemover(handler, cr)), cleanerExecutor)
+                          .exceptionally(t -> {
+                            LOG.error("Error clearing: {}", cr.getFullPartitionName(), t);
+                            return null;
+                          });

Review Comment:
   Implemented it in a similar way. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {

Review Comment:
   The other cleanup request types that will be added is - Abort request type. 
   `dropPartition` seems to be too narrow a use case (hence thought a separate type was not required)  and is handled within the compaction request itself.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {

Review Comment:
   The other cleanup request type that will be added is - Abort request type. 
   `dropPartition` seems to be too narrow a use case (hence thought a separate type was not required)  and is handled within the compaction request itself.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -122,11 +80,8 @@ public void run() {
     try {
       do {
         TxnStore.MutexAPI.LockHandle handle = null;
-        invalidateMetaCache();
+        cleaningRequestHandlers.forEach(CleaningRequestHandler::invalidateMetaCache);

Review Comment:
   Added a class called CacheContainer and created an object from it. Used this object for handling all cache related operations amongst all handlers. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +46,28 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<RequestHandler> requestHandlers;
 
   @Override
   public void init(AtomicBoolean stop) throws Exception {
     super.init(stop);
-    replChangeManager = ReplChangeManager.getInstance(conf);
     checkInterval = conf.getTimeVar(
             HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, TimeUnit.MILLISECONDS);
-    cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
-            conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_THREADS_NUM),
-            COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
     metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
         MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
-    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,
-            MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
-    initializeCache(tableCacheOn);
+    if (CollectionUtils.isEmpty(requestHandlers)) {

Review Comment:
   Why do we need this check? it's a bit confusing initialization. I would assume this is required for tests? 



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  protected final MetadataCache metadataCache;
+  protected final FSRemover fsRemover;
+  protected final ExecutorService cleanerExecutor;
+
+  RequestHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
+                         boolean metricsEnabled, FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    this.metadataCache = metadataCache;
+    this.metricsEnabled = metricsEnabled;
+    this.fsRemover = fsRemover;
+    this.cleanerExecutor = cleanerExecutor;
+  }
+
+  protected abstract List<Runnable> fetchCleanTasks() throws MetaException;
+
+  public void process() throws Exception {

Review Comment:
   `execute`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  protected final MetadataCache metadataCache;
+  protected final FSRemover fsRemover;
+  protected final ExecutorService cleanerExecutor;
+
+  RequestHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
+                         boolean metricsEnabled, FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    this.metadataCache = metadataCache;
+    this.metricsEnabled = metricsEnabled;
+    this.fsRemover = fsRemover;
+    this.cleanerExecutor = cleanerExecutor;
+  }
+
+  protected abstract List<Runnable> fetchCleanTasks() throws MetaException;
+
+  public void process() throws Exception {

Review Comment:
   i think this class should return the list of tasks to the Cleaner but Cleaner should execute them. Also, no need to supply the fsRemover 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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private final String runAs;
+  private final String cleanerMetric;

Review Comment:
   I have refactored such that cleaner metric is handled within the handler. Hence removed it from Cleaning request.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionHandler.java:
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+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.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CacheContainer;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest.CleaningRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+import static java.util.Objects.isNull;
+
+/**
+ * A compaction based implementation of RequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionHandler extends RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionHandler.class.getName());
+
+  public CompactionHandler(HiveConf conf, TxnStore txnHandler,
+                                          CacheContainer cacheContainer, boolean metricsEnabled,
+                                          FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    super(conf, txnHandler, cacheContainer, metricsEnabled, fsRemover, cleanerExecutor);
+  }
+
+  @Override
+  protected void processInternal() throws Exception {
+    long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
+    long retentionTime = HiveConf.getBoolVar(conf, HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED)
+            ? HiveConf.getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETENTION_TIME, TimeUnit.MILLISECONDS)
+            : 0;
+    List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
+    if (!readyToClean.isEmpty()) {
+      long minTxnIdSeenOpen = txnHandler.findMinTxnIdSeenOpen();
+      final long cleanerWaterMark =
+              minTxnIdSeenOpen < 0 ? minOpenTxnId : Math.min(minOpenTxnId, minTxnIdSeenOpen);
+
+      LOG.info("Cleaning based on min open txn id: {}", cleanerWaterMark);
+      // For checking which compaction can be cleaned we can use the minOpenTxnId
+      // However findReadyToClean will return all records that were compacted with old version of HMS
+      // where the CQ_NEXT_TXN_ID is not set. For these compactions we need to provide minTxnIdSeenOpen
+      // to the clean method, to avoid cleaning up deltas needed for running queries
+      // when min_history_level is finally dropped, than every HMS will commit compaction the new way
+      // and minTxnIdSeenOpen can be removed and minOpenTxnId can be used instead.
+      List<CompletableFuture<Void>> asyncTasks = new ArrayList<>();
+      for (CompactionInfo ci : readyToClean) {
+        CompletableFuture<Void> asyncTask = CompletableFuture.runAsync(
+                CompactorUtil.ThrowingRunnable.unchecked(() -> clean(ci, cleanerWaterMark, metricsEnabled)), cleanerExecutor)
+                .exceptionally(t -> {
+                  LOG.error("Error clearing {} due to :", ci.getFullPartitionName(), t);
+                  return null;
+                });
+        asyncTasks.add(asyncTask);
+      }
+      //Use get instead of join, so we can receive InterruptedException and shutdown gracefully
+      CompletableFuture.allOf(asyncTasks.toArray(new CompletableFuture[0])).get();
+    }
+  }
+
+  private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", ci);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
+            (!isNull(ci.type) ? ci.type.toString().toLowerCase() : null);
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(CompactionHandler.class.getName(), cleanerMetric);
+      }
+      final String location = ci.getProperty("location");
+
+      Table t = null;
+      Partition p = null;
+
+      if (isNull(location)) {
+        t = cacheContainer.computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.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. {}", ci.getFullTableName(),
+                  idWatermark(ci));
+          txnHandler.markCleaned(ci);
+          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", ci.getFullTableName());
+          txnHandler.markCleaned(ci);
+          return;
+        }
+        if (!isNull(ci.partName)) {
+          p = resolvePartition(ci.dbname, ci.tableName, ci.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. {}",
+                    ci.getFullPartitionName(), idWatermark(ci));
+            txnHandler.markCleaned(ci);
+            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", ci.getFullPartitionName());
+            txnHandler.markCleaned(ci);
+            return;
+          }
+        }
+      }
+      txnHandler.markCleanerStart(ci);
+
+      if (!isNull(t) || !isNull(ci.partName)) {
+        String path = isNull(location)
+                ? CompactorUtil.resolveStorageDescriptor(t, p).getLocation()
+                : location;
+        boolean dropPartition = !isNull(ci.partName) && isNull(p);
+
+        //check if partition wasn't re-created
+        if (dropPartition && isNull(resolvePartition(ci.dbname, ci.tableName, ci.partName))) {
+          cleanUsingLocation(ci, path, true);
+        } else {
+          cleanUsingAcidDir(ci, path, minOpenTxnGLB);
+        }
+      } else {
+        cleanUsingLocation(ci, location, false);
+      }
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", ci,
+              e.getMessage());
+      ci.errorMessage = e.getMessage();
+      if (metricsEnabled) {
+        Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
+      }
+      handleCleanerAttemptFailure(ci);
+    }  finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(CompactionHandler.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void cleanUsingLocation(CompactionInfo ci, String path, boolean requiresLock) throws MetaException {
+    List<Path> deleted;
+    if (requiresLock) {
+      LockRequest lockRequest = createLockRequest(ci, 0, LockType.EXCL_WRITE, DataOperationType.DELETE);
+      LockResponse res = null;
+      try {
+        res = txnHandler.lock(lockRequest);
+        deleted = fsRemover.clean(getCleaningRequestBasedOnLocation(ci, path));
+      } catch (NoSuchTxnException | TxnAbortedException e) {
+        LOG.error("Error while trying to acquire exclusive write lock: {}", e.getMessage());
+        throw new MetaException(e.getMessage());
+      } finally {
+        if (res != null) {
+          try {
+            txnHandler.unlock(new UnlockRequest(res.getLockid()));
+          } catch (NoSuchLockException | TxnOpenException e) {
+            LOG.error("Error while trying to release exclusive write lock: {}", e.getMessage());
+          }
+        }
+      }
+    } else {
+      deleted = fsRemover.clean(getCleaningRequestBasedOnLocation(ci, path));
+    }
+    if (!deleted.isEmpty()) {
+      txnHandler.markCleaned(ci);
+    } else {
+      txnHandler.clearCleanerStart(ci);
+    }
+  }
+
+  private void cleanUsingAcidDir(CompactionInfo ci, String location, long minOpenTxnGLB) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+    /*
+     * {@code validTxnList} is capped by minOpenTxnGLB so if
+     * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
+     * produced by a compactor, that means every reader that could be active right now see it
+     * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
+     * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
+     *
+     *
+     * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
+     * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
+     * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
+     * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
+     * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
+     * writeId:17.  Compactor will produce base_17_c160.
+     * Suppose txnid:149 writes delta_18_18
+     * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
+     * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
+     * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
+     * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
+     * metadata that says that 18 is aborted.
+     * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
+     * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
+     * it (since it has nothing but aborted data).  But we can't tell which actually happened
+     * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
+     *
+     * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
+     * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
+     * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
+     * useful if there is all of a sudden a flood of aborted txns.  (For another day).
+     */
+
+    // 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);
+    Table table = cacheContainer.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 CleaningRequestBuilder().setLocation(location).setRunAs(ci.runAs)
+            .setObsoleteDirs(obsoleteDirs).setPurge(true).setFullPartitionName(ci.getFullPartitionName())
+            .build());
+
+    if (deleted.size() > 0) {
+      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);
+
+    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(idWatermark(ci) + " All cleared below the watermark: " + ci.highestWriteId + " from " + location);
+      success = true;
+    }
+    if (success || CompactorUtil.isDynPartAbort(table, ci.partName)) {
+      txnHandler.markCleaned(ci);
+    } else {
+      txnHandler.clearCleanerStart(ci);
+      LOG.warn("No files were removed. Leaving queue entry {} in ready for cleaning state.", ci);
+    }
+  }
+
+  protected LockRequest createLockRequest(CompactionInfo ci, long txnId, LockType lockType, DataOperationType opType) {

Review Comment:
   Makes sense. Removed them. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HandlerFactory.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CompactionHandler;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.Handler;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A factory class to fetch handlers.
+ */
+public class HandlerFactory {

Review Comment:
   Made CompactionHandler package private and moved HandlerFactory to handler package. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionCleaningRequest.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+
+import java.util.Map;
+
+/**
+ * A cleaning request class specific to compaction based cleanup.
+ */
+public class CompactionCleaningRequest extends CleaningRequest {
+
+  private final CompactionInfo compactionInfo;
+  private final Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots;
+
+  public CompactionCleaningRequest(CompactionCleaningRequestBuilder builder) {

Review Comment:
   build() function initiates creation of object with the information that builder provides. Its just builder design pattern.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private final String runAs;
+  private final String cleanerMetric;
+  private final String dbName;
+  private final String tableName;
+  private final String partitionName;
+  private final boolean dropPartition;
+  private final String fullPartitionName;
+
+  public CleaningRequest(CleaningRequestBuilder<? extends CleaningRequestBuilder<?>> builder) {
+    this.type = builder.type;
+    this.location = builder.location;
+    this.obsoleteDirs = builder.obsoleteDirs;
+    this.purge = builder.purge;
+    this.fs = builder.fs;
+    this.runAs = builder.runAs;
+    this.cleanerMetric = builder.cleanerMetric;
+    this.dbName = builder.dbName;

Review Comment:
   The idea is generic cleaning request class should not have information on compaction.
   CompactionCleaningRequest will have the compaction info object. Hence we need to pass this information to generic cleaning request.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CacheContainer.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class CacheContainer {

Review Comment:
   I am not good with naming, but maybe call it `MetadataCache`?



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionHandler.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+
+/**
+ * A compaction based implementation of Handler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+public class CompactionHandler extends Handler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionHandler.class.getName());
+
+  public CompactionHandler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    super(conf, txnHandler, metricsEnabled);
+  }
+
+  @Override
+  public List<CleaningRequest> findReadyToClean() throws MetaException {
+    List<CleaningRequest> cleaningRequests = new ArrayList<>();
+    long retentionTime = HiveConf.getBoolVar(conf, HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED)
+            ? HiveConf.getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETENTION_TIME, TimeUnit.MILLISECONDS)
+            : 0;
+    long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
+    List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
+
+    if (!readyToClean.isEmpty()) {
+      long minTxnIdSeenOpen = txnHandler.findMinTxnIdSeenOpen();
+      final long cleanerWaterMark =
+              minTxnIdSeenOpen < 0 ? minOpenTxnId : Math.min(minOpenTxnId, minTxnIdSeenOpen);
+
+      LOG.info("Cleaning based on min open txn id: {}", cleanerWaterMark);
+      // For checking which compaction can be cleaned we can use the minOpenTxnId
+      // However findReadyToClean will return all records that were compacted with old version of HMS
+      // where the CQ_NEXT_TXN_ID is not set. For these compactions we need to provide minTxnIdSeenOpen
+      // to the clean method, to avoid cleaning up deltas needed for running queries
+      // when min_history_level is finally dropped, than every HMS will commit compaction the new way
+      // and minTxnIdSeenOpen can be removed and minOpenTxnId can be used instead.
+      for (CompactionInfo ci : readyToClean) {
+        LOG.info("Starting cleaning for {}", ci);
+        try {
+          final String location = ci.getProperty("location");
+
+          Table t = null;
+          Partition p = null;
+
+          if (location == null) {
+            t = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.tableName));
+            if (t == null) {
+              // The table was dropped before we got around to cleaning it.
+              LOG.info("Unable to find table {}, assuming it was dropped. {}", ci.getFullTableName(),
+                      idWatermark(ci));
+              txnHandler.markCleaned(ci);
+              continue;
+            }
+            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", ci.getFullTableName());
+              txnHandler.markCleaned(ci);
+              continue;
+            }
+            if (ci.partName != null) {
+              p = resolvePartition(ci.dbname, ci.tableName, ci.partName);
+              if (p == null) {
+                // The partition was dropped before we got around to cleaning it.
+                LOG.info("Unable to find partition {}, assuming it was dropped. {}", ci.getFullTableName(), idWatermark(ci));
+                txnHandler.markCleaned(ci);
+                continue;
+              }
+              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", ci.getFullPartitionName());
+                txnHandler.markCleaned(ci);
+                continue;
+              }
+            }
+          }
+
+          if (t != null || ci.partName != null) {
+            String path = location == null
+                    ? CompactorUtil.resolveStorageDescriptor(t, p).getLocation()
+                    : location;
+            boolean dropPartition = ci.partName != null && p == null;
+            if (dropPartition) {
+              //check if partition wasn't re-created
+              if (resolvePartition(ci.dbname, ci.tableName, ci.partName) == null) {
+                String strIfPurge = ci.getProperty("ifPurge");
+                boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
+
+                Path obsoletePath = new Path(path);
+                cleaningRequests.add(new CompactionCleaningRequest(path, ci, Collections.singletonList(obsoletePath), ifPurge,
+                        obsoletePath.getFileSystem(conf), null, true));
+                continue;
+              }
+            }
+
+            ValidTxnList validTxnList =
+                    TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), cleanerWaterMark);
+            //save it so that getAcidState() sees it
+            conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+            /*
+             * {@code validTxnList} is capped by minOpenTxnGLB so if
+             * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
+             * produced by a compactor, that means every reader that could be active right now see it
+             * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
+             * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
+             *
+             *
+             * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
+             * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
+             * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
+             * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
+             * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
+             * writeId:17.  Compactor will produce base_17_c160.
+             * Suppose txnid:149 writes delta_18_18
+             * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
+             * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
+             * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
+             * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
+             * metadata that says that 18 is aborted.
+             * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
+             * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
+             * it (since it has nothing but aborted data).  But we can't tell which actually happened
+             * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
+             *
+             * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
+             * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
+             * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
+             * useful if there is all of a sudden a flood of aborted txns.  (For another day).
+             */
+
+            // 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 loc = new Path(path);
+            FileSystem fs = loc.getFileSystem(conf);
+
+            // Collect all the files/dirs
+            Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, loc);
+            AcidDirectory dir = AcidUtils.getAcidState(fs, loc, conf, validWriteIdList, Ref.from(false), false,
+                    dirSnapshots);
+            Table table = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.tableName));
+            boolean isDynPartAbort = CompactorUtil.isDynPartAbort(table, ci.partName);
+
+            List<Path> obsoleteDirs = getObsoleteDirs(dir, isDynPartAbort);
+            if (isDynPartAbort || dir.hasUncompactedAborts()) {
+              ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
+            }
+
+            cleaningRequests.add(new CompactionCleaningRequest(path, ci, obsoleteDirs, true, fs, dirSnapshots, false));
+          } else {
+            String strIfPurge = ci.getProperty("ifPurge");
+            boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
+
+            Path obsoletePath = new Path(location);
+            cleaningRequests.add(new CompactionCleaningRequest(location, ci, Collections.singletonList(obsoletePath), ifPurge,
+                    obsoletePath.getFileSystem(conf), null, false));
+          }
+        } catch (Exception e) {
+          LOG.warn("Cleaning request was not successful generated for : {} due to {}", idWatermark(ci), e.getMessage());
+          ci.errorMessage = e.getMessage();
+          if (metricsEnabled) {
+            Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
+          }
+          handleCleanerAttemptFailure(ci);
+        }
+      }
+    }
+    return cleaningRequests;
+  }
+
+  @Override
+  public void beforeExecutingCleaningRequest(CleaningRequest cleaningRequest) throws MetaException {
+    CompactionInfo ci = ((CompactionCleaningRequest) cleaningRequest).getCompactionInfo();
+    txnHandler.markCleanerStart(ci);
+  }
+
+  @Override
+  public void afterExecutingCleaningRequest(CleaningRequest cleaningRequest, List<Path> deletedFiles) throws MetaException {
+    CompactionInfo ci = ((CompactionCleaningRequest) cleaningRequest).getCompactionInfo();
+    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = ((CompactionCleaningRequest) cleaningRequest).getHdfsDirSnapshots();
+    // Make sure there are no leftovers below the compacted watermark
+    if (dirSnapshots != null) {
+      conf.set(ValidTxnList.VALID_TXNS_KEY, new ValidReadTxnList().toString());
+      Path path = new Path(cleaningRequest.getLocation());
+      Table table;
+      boolean success = false;

Review Comment:
   Made afterExecutingCleaningRequest() return a boolean to notify FSRemover whether the cleanup was successful 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler<T extends CleaningRequest> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Handler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  private Optional<Cache<String, TBase>> metaCache;
+
+  Handler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    boolean tableCacheOn = MetastoreConf.getBoolVar(this.conf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
+    this.metaCache = initializeCache(tableCacheOn);
+    this.metricsEnabled = metricsEnabled;
+  }
+
+  public HiveConf getConf() {
+    return conf;
+  }
+
+  public TxnStore getTxnHandler() {
+    return txnHandler;
+  }
+
+  public boolean isMetricsEnabled() {
+    return metricsEnabled;
+  }
+
+  /**
+   * Find the list of objects which are ready for cleaning.
+   * @return Cleaning requests
+   */
+  public abstract List<T> findReadyToClean() throws MetaException;
+
+  /**
+   * Execute just before cleanup
+   * @param cleaningRequest - Cleaning request
+   */
+  public abstract void beforeExecutingCleaningRequest(T cleaningRequest) throws MetaException;
+
+  /**
+   * Execute just after cleanup
+   * @param cleaningRequest Cleaning request
+   * @param deletedFiles List of deleted files
+   * @return True if cleanup was successful, false otherwise
+   * @throws MetaException
+   */
+  public abstract boolean afterExecutingCleaningRequest(T cleaningRequest, List<Path> deletedFiles) throws MetaException;
+
+  /**
+   * Execute in the event of failure
+   * @param cleaningRequest Cleaning request
+   * @param ex Failure exception
+   * @throws MetaException
+   */
+  public abstract void failureExecutingCleaningRequest(T cleaningRequest, Exception ex) throws MetaException;
+
+  public Table resolveTable(String dbName, String tableName) throws MetaException {
+    try {
+      return getMSForConf(conf).getTable(getDefaultCatalog(conf), dbName, tableName);
+    } catch (MetaException e) {
+      LOG.error("Unable to find table {}.{}, {}", dbName, tableName, e.getMessage());
+      throw e;
+    }
+  }
+
+  protected Partition resolvePartition(String dbName, String tableName, String partName) throws MetaException {
+    if (partName != null) {
+      List<Partition> parts;
+      try {
+        parts = CompactorUtil.getPartitionsByNames(conf, dbName, tableName, partName);
+        if (parts == null || parts.isEmpty()) {
+          // The partition got dropped before we went looking for it.
+          return null;
+        }
+      } catch (Exception e) {
+        LOG.error("Unable to find partition: {}.{}.{}", dbName, tableName, partName, e);
+        throw e;
+      }
+      if (parts.size() != 1) {
+        LOG.error("{}.{}.{} does not refer to a single partition. {}", dbName, tableName, partName,
+                Arrays.toString(parts.toArray()));
+        throw new MetaException(String.join("Too many partitions for : ", dbName, tableName, partName));
+      }
+      return parts.get(0);
+    } else {
+      return null;
+    }
+  }
+
+  public <B extends TBase<B,?>> B computeIfAbsent(String key, Callable<B> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (B) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {
+    if (tableCacheOn) {
+      metaCache = Optional.of(CacheBuilder.newBuilder().softValues().build());
+    }
+    return metaCache;
+  }
+
+  public void invalidateMetaCache() {
+    metaCache.ifPresent(Cache::invalidateAll);
+  }
+
+  protected List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
+    List<Path> obsoleteDirs = dir.getObsolete();
+    /*
+     * add anything in 'dir'  that only has data from aborted transactions - no one should be
+     * trying to read anything in that dir (except getAcidState() that only reads the name of
+     * this dir itself)
+     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
+     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
+     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
+     * See {@link TxnStore#markCleaned(CompactionInfo)}
+     */
+    obsoleteDirs.addAll(dir.getAbortedDirectories());
+    if (isDynPartAbort) {
+      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
+      // Including obsolete directories for partitioned tables can result in data loss.
+      obsoleteDirs = dir.getAbortedDirectories();
+    }
+    return obsoleteDirs;
+  }

Review Comment:
   Moved `getObsoleteDirs` to CompactorUtil since its using AcidDirectory which might not be generic to all handlers. However, the caching is done on a handler level now. Hence, caching functions are kept here itself.
   `resolvePartition` seems generic to most handlers, hence kept it 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] veghlaci05 commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.Handler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in handler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover implements ThrowingRunnable<MetaException> {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Handler handler;
+  private final CleaningRequest cr;
+  private final ReplChangeManager replChangeManager;
+
+  public FSRemover(Handler handler, CleaningRequest cr) throws MetaException {
+    this.handler = handler;
+    this.cr = cr;
+    this.replChangeManager = ReplChangeManager.getInstance(this.handler.getConf());

Review Comment:
   This should be passed as a constructor argument



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

Review Comment:
   Cache is initialized based on a config - 
   In the current master this is code - 
   https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java#L114-L116
   
   I think accepting a boolean in constructor and initializing might also make sense. Return type is also not used anywhere. 



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   @deniskuzZ Optional is required to check whether the cache is actually built 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandlerFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A factory class to fetch handlers.
+ */
+public class RequestHandlerFactory {
+  private static final RequestHandlerFactory INSTANCE = new RequestHandlerFactory();
+
+  public static RequestHandlerFactory getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Factory class, no need to expose constructor.
+   */
+  private RequestHandlerFactory() {
+  }
+
+  public List<RequestHandler> getHandlers(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,

Review Comment:
   `List<Runnable> getTasks`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java:
##########
@@ -97,16 +90,11 @@ public void init(AtomicBoolean stop) throws Exception {
   }
 
   @Override List<Partition> getPartitionsByNames(CompactionInfo ci) throws MetaException {
-    try {
-      return getMSForConf(conf).getPartitionsByNames(getDefaultCatalog(conf), ci.dbname, ci.tableName,
-          Collections.singletonList(ci.partName));
-    } catch (MetaException e) {
-      LOG.error("Unable to get partitions by name for CompactionInfo=" + ci);
-      throw e;
-    } catch (NoSuchObjectException e) {
-      LOG.error("Unable to get partitions by name for CompactionInfo=" + ci);
-      throw new MetaException(e.toString());
-    }
+    return CompactorUtil.getPartitionsByNames(conf, ci.dbname, ci.tableName, ci.partName);
+  }
+
+  public boolean isCacheEnabled() {

Review Comment:
   shouldn't it be abstract?



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionHandler.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+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.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CacheContainer;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest.CleaningRequestBuilder;
+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.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+import static java.util.Objects.isNull;
+
+/**
+ * A compaction based implementation of RequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionHandler extends RequestHandler {

Review Comment:
   i think it should be either under the cleaner package or have `Cleaner` in the class name



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -212,323 +140,8 @@ public void run() {
     }
   }
 
-  private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
-    LOG.info("Starting cleaning for " + ci);
-    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
-    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
-        (ci.type != null ? ci.type.toString().toLowerCase() : null);
-    try {
-      if (metricsEnabled) {
-        perfLogger.perfLogBegin(CLASS_NAME, cleanerMetric);
-      }
-      final String location = ci.getProperty("location");
-
-      Callable<Boolean> cleanUpTask;
-      Table t = null;
-      Partition p = null;
-
-      if (location == null) {
-        t = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-        if (t == null) {
-          // The table was dropped before we got around to cleaning it.
-          LOG.info("Unable to find table " + ci.getFullTableName() + ", assuming it was dropped." +
-            idWatermark(ci));
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
-          // The table was marked no clean up true.
-          LOG.info("Skipping table " + ci.getFullTableName() + " clean up, as NO_CLEANUP set to true");
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (ci.partName != null) {
-          p = resolvePartition(ci);
-          if (p == null) {
-            // The partition was dropped before we got around to cleaning it.
-            LOG.info("Unable to find partition " + ci.getFullPartitionName() +
-              ", assuming it was dropped." + idWatermark(ci));
-            txnHandler.markCleaned(ci);
-            return;
-          }
-          if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
-            // The partition was marked no clean up true.
-            LOG.info("Skipping partition " + ci.getFullPartitionName() + " clean up, as NO_CLEANUP set to true");
-            txnHandler.markCleaned(ci);
-            return;
-          }
-        }
-      }
-      txnHandler.markCleanerStart(ci);
-
-      if (t != null || ci.partName != null) {
-        String path = location == null
-            ? resolveStorageDescriptor(t, p).getLocation()
-            : location;
-        boolean dropPartition = ci.partName != null && p == null;
-        cleanUpTask = () -> removeFiles(path, minOpenTxnGLB, ci, dropPartition);
-      } else {
-        cleanUpTask = () -> removeFiles(location, ci);
-      }
-
-      Ref<Boolean> removedFiles = Ref.from(false);
-      if (runJobAsSelf(ci.runAs)) {
-        removedFiles.value = cleanUpTask.call();
-      } else {
-        LOG.info("Cleaning as user " + ci.runAs + " for " + ci.getFullPartitionName());
-        UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs,
-            UserGroupInformation.getLoginUser());
-        try {
-          ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-            removedFiles.value = cleanUpTask.call();
-            return null;
-          });
-        } finally {
-          try {
-            FileSystem.closeAllForUGI(ugi);
-          } catch (IOException exception) {
-            LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " +
-                ci.getFullPartitionName() + idWatermark(ci), exception);
-          }
-        }
-      }
-      if (removedFiles.value || isDynPartAbort(t, ci)) {
-        txnHandler.markCleaned(ci);
-      } else {
-        txnHandler.clearCleanerStart(ci);
-        LOG.warn("No files were removed. Leaving queue entry " + ci + " in ready for cleaning state.");
-      }
-    } catch (Exception e) {
-      LOG.error("Caught exception when cleaning, unable to complete cleaning of " + ci + " " +
-          StringUtils.stringifyException(e));
-      ci.errorMessage = e.getMessage();
-      if (metricsEnabled) {
-        Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
-      }
-      handleCleanerAttemptFailure(ci);
-    }  finally {
-      if (metricsEnabled) {
-        perfLogger.perfLogEnd(CLASS_NAME, cleanerMetric);
-      }
-    }
-  }
-
-  private void handleCleanerAttemptFailure(CompactionInfo ci) throws MetaException {
-    long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS);
-    int cleanAttempts = 0;
-    if (ci.retryRetention > 0) {
-      cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1;
-    }
-    if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) {
-      //Mark it as failed if the max attempt threshold is reached.
-      txnHandler.markFailed(ci);
-    } else {
-      //Calculate retry retention time and update record.
-      ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention;
-      txnHandler.setCleanerRetryRetentionTimeOnError(ci);
-    }
-  }
-
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)
-      throws NoSuchTxnException, MetaException {
-    List<String> tblNames = Collections.singletonList(AcidUtils.getFullTableName(ci.dbname, ci.tableName));
-    GetValidWriteIdsRequest request = new GetValidWriteIdsRequest(tblNames);
-    request.setValidTxnList(validTxnList.writeToString());
-    GetValidWriteIdsResponse rsp = txnHandler.getValidWriteIds(request);
-    // we could have no write IDs for a table if it was never written to but
-    // since we are in the Cleaner phase of compactions, there must have
-    // been some delta/base dirs
-    assert rsp != null && rsp.getTblValidWriteIdsSize() == 1;
-    ValidReaderWriteIdList validWriteIdList =
-        TxnCommonUtils.createValidReaderWriteIdList(rsp.getTblValidWriteIds().get(0));
-    /*
-     * We need to filter the obsoletes dir list, to only remove directories that were made obsolete by this compaction
-     * If we have a higher retentionTime it is possible for a second compaction to run on the same partition. Cleaning up the first compaction
-     * should not touch the newer obsolete directories to not to violate the retentionTime for those.
-     */
-    if (ci.highestWriteId < validWriteIdList.getHighWatermark()) {
-      validWriteIdList = validWriteIdList.updateHighWatermark(ci.highestWriteId);
-    }
-    return validWriteIdList;
-  }
-
-  private static boolean isDynPartAbort(Table t, CompactionInfo ci) {
-    return Optional.ofNullable(t).map(Table::getPartitionKeys).filter(pk -> pk.size() > 0).isPresent()
-        && ci.partName == null;
-  }
-
-  private static String idWatermark(CompactionInfo ci) {
-    return " id=" + ci.id;
-  }
-
-  private boolean removeFiles(String location, long minOpenTxnGLB, CompactionInfo ci, boolean dropPartition)
-      throws Exception {
-
-    if (dropPartition) {
-      LockRequest lockRequest = createLockRequest(ci, 0, LockType.EXCL_WRITE, DataOperationType.DELETE);
-      LockResponse res = null;
-
-      try {
-        res = txnHandler.lock(lockRequest);
-        if (res.getState() == LockState.ACQUIRED) {
-          //check if partition wasn't re-created
-          if (resolvePartition(ci) == null) {
-            return removeFiles(location, ci);
-          }
-        }
-      } catch (NoSuchTxnException | TxnAbortedException e) {
-        LOG.error(e.getMessage());
-      } finally {
-        if (res != null) {
-          try {
-            txnHandler.unlock(new UnlockRequest(res.getLockid()));
-          } catch (NoSuchLockException | TxnOpenException e) {
-            LOG.error(e.getMessage());
-          }
-        }
-      }
-    }
-
-    ValidTxnList validTxnList =
-      TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB);
-    //save it so that getAcidState() sees it
-    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
-    /**
-     * {@code validTxnList} is capped by minOpenTxnGLB so if
-     * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
-     * produced by a compactor, that means every reader that could be active right now see it
-     * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
-     * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
-     *
-     *
-     * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
-     * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
-     * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
-     * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
-     * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
-     * writeId:17.  Compactor will produce base_17_c160.
-     * Suppose txnid:149 writes delta_18_18
-     * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
-     * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
-     * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
-     * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
-     * metadata that says that 18 is aborted.
-     * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
-     * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
-     * it (since it has nothing but aborted data).  But we can't tell which actually happened
-     * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
-     *
-     * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
-     * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
-     * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
-     * useful if there is all of a sudden a flood of aborted txns.  (For another day).
-     */
-
-    // 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);
-
-    return removeFiles(location, validWriteIdList, ci);
-  }
-  /**
-   * @return true if any files were removed
-   */
-  private boolean removeFiles(String location, ValidWriteIdList writeIdList, CompactionInfo ci)
-      throws Exception {
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-    
-    // Collect all of the files/dirs
-    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, path);
-    AcidDirectory dir = AcidUtils.getAcidState(fs, path, conf, writeIdList, Ref.from(false), false, 
-        dirSnapshots);
-    Table table = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-    boolean isDynPartAbort = isDynPartAbort(table, ci);
-    
-    List<Path> obsoleteDirs = getObsoleteDirs(dir, isDynPartAbort);
-    if (isDynPartAbort || dir.hasUncompactedAborts()) {
-      ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
-    }
-    List<Path> deleted = remove(location, ci, obsoleteDirs, true, fs);
-    if (dir.getObsolete().size() > 0) {
-      AcidMetricService.updateMetricsFromCleaner(ci.dbname, ci.tableName, ci.partName, dir.getObsolete(), conf,
-          txnHandler);
-    }
-    // Make sure there are no leftovers below the compacted watermark
-    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);
-    
-    List<Path> remained = subtract(getObsoleteDirs(dir, isDynPartAbort), deleted);
-    if (!remained.isEmpty()) {
-      LOG.warn(idWatermark(ci) + " Remained " + remained.size() +
-        " obsolete directories from " + location + ". " + getDebugInfo(remained));
-      return false;
-    }
-    LOG.debug(idWatermark(ci) + " All cleared below the watermark: " + ci.highestWriteId + " from " + location);
-    return true;
-  }
-  
-  private List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
-    List<Path> obsoleteDirs = dir.getObsolete();
-    /**
-     * add anything in 'dir'  that only has data from aborted transactions - no one should be
-     * trying to read anything in that dir (except getAcidState() that only reads the name of
-     * this dir itself)
-     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
-     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
-     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
-     * See {@link TxnStore#markCleaned(CompactionInfo)}
-     */
-    obsoleteDirs.addAll(dir.getAbortedDirectories());
-    if (isDynPartAbort) {
-      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
-      // Including obsolete directories for partitioned tables can result in data loss.
-      obsoleteDirs = dir.getAbortedDirectories();
-    }
-    return obsoleteDirs;
-  }
-
-  private boolean removeFiles(String location, CompactionInfo ci) throws IOException, MetaException {
-    String strIfPurge = ci.getProperty("ifPurge");
-    boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
-    
-    Path path = new Path(location);
-    return !remove(location, ci, Collections.singletonList(path), ifPurge,
-      path.getFileSystem(conf)).isEmpty();
-  }
-
-  private List<Path> remove(String location, CompactionInfo ci, List<Path> paths, boolean ifPurge, FileSystem fs)
-      throws MetaException, IOException {
-    List<Path> deleted = new ArrayList<>();
-    if (paths.size() < 1) {
-      return deleted;
-    }
-    LOG.info(idWatermark(ci) + " About to remove " + paths.size() +
-      " obsolete directories from " + location + ". " + getDebugInfo(paths));
-    boolean needCmRecycle;
-    try {
-      Database db = getMSForConf(conf).getDatabase(getDefaultCatalog(conf), ci.dbname);
-      needCmRecycle = ReplChangeManager.isSourceOfReplication(db);
-    } catch (NoSuchObjectException ex) {
-      // can not drop a database which is a source of replication
-      needCmRecycle = false;
-    }
-    for (Path dead : paths) {
-      LOG.debug("Going to delete path " + dead.toString());
-      if (needCmRecycle) {
-        replChangeManager.recycle(dead, ReplChangeManager.RecycleType.MOVE, ifPurge);
-      }
-      if (FileUtils.moveToTrash(fs, dead, conf, ifPurge)) {
-        deleted.add(dead);
-      }
-    }
-    return deleted;
-  }
-  
-  private String getDebugInfo(List<Path> paths) {
-    return "[" + paths.stream().map(Path::getName).collect(Collectors.joining(",")) + ']';
+  public void setHandlers(List<Handler> handlers) {

Review Comment:
   Added default constructors & a constructor which takes in Handlers. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler<T extends CleaningRequest> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Handler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  private Optional<Cache<String, TBase>> metaCache;
+
+  Handler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    boolean tableCacheOn = MetastoreConf.getBoolVar(this.conf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
+    this.metaCache = initializeCache(tableCacheOn);
+    this.metricsEnabled = metricsEnabled;
+  }
+
+  public HiveConf getConf() {
+    return conf;
+  }
+
+  public TxnStore getTxnHandler() {
+    return txnHandler;
+  }
+
+  public boolean isMetricsEnabled() {
+    return metricsEnabled;
+  }
+
+  /**
+   * Find the list of objects which are ready for cleaning.
+   * @return Cleaning requests
+   */
+  public abstract List<T> findReadyToClean() throws MetaException;
+
+  /**
+   * Execute just before cleanup
+   * @param cleaningRequest - Cleaning request
+   */
+  public abstract void beforeExecutingCleaningRequest(T cleaningRequest) throws MetaException;
+
+  /**
+   * Execute just after cleanup
+   * @param cleaningRequest Cleaning request
+   * @param deletedFiles List of deleted files
+   * @return True if cleanup was successful, false otherwise
+   * @throws MetaException
+   */
+  public abstract boolean afterExecutingCleaningRequest(T cleaningRequest, List<Path> deletedFiles) throws MetaException;
+
+  /**
+   * Execute in the event of failure
+   * @param cleaningRequest Cleaning request
+   * @param ex Failure exception
+   * @throws MetaException
+   */
+  public abstract void failureExecutingCleaningRequest(T cleaningRequest, Exception ex) throws MetaException;
+
+  public Table resolveTable(String dbName, String tableName) throws MetaException {
+    try {
+      return getMSForConf(conf).getTable(getDefaultCatalog(conf), dbName, tableName);
+    } catch (MetaException e) {
+      LOG.error("Unable to find table {}.{}, {}", dbName, tableName, e.getMessage());
+      throw e;
+    }
+  }
+
+  protected Partition resolvePartition(String dbName, String tableName, String partName) throws MetaException {
+    if (partName != null) {
+      List<Partition> parts;
+      try {
+        parts = CompactorUtil.getPartitionsByNames(conf, dbName, tableName, partName);
+        if (parts == null || parts.isEmpty()) {
+          // The partition got dropped before we went looking for it.
+          return null;
+        }
+      } catch (Exception e) {
+        LOG.error("Unable to find partition: {}.{}.{}", dbName, tableName, partName, e);
+        throw e;
+      }
+      if (parts.size() != 1) {
+        LOG.error("{}.{}.{} does not refer to a single partition. {}", dbName, tableName, partName,
+                Arrays.toString(parts.toArray()));
+        throw new MetaException(String.join("Too many partitions for : ", dbName, tableName, partName));
+      }
+      return parts.get(0);
+    } else {
+      return null;
+    }
+  }
+
+  public <B extends TBase<B,?>> B computeIfAbsent(String key, Callable<B> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (B) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {
+    if (tableCacheOn) {
+      metaCache = Optional.of(CacheBuilder.newBuilder().softValues().build());
+    }
+    return metaCache;
+  }
+
+  public void invalidateMetaCache() {
+    metaCache.ifPresent(Cache::invalidateAll);
+  }
+
+  protected List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
+    List<Path> obsoleteDirs = dir.getObsolete();
+    /*
+     * add anything in 'dir'  that only has data from aborted transactions - no one should be
+     * trying to read anything in that dir (except getAcidState() that only reads the name of
+     * this dir itself)
+     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
+     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
+     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
+     * See {@link TxnStore#markCleaned(CompactionInfo)}
+     */
+    obsoleteDirs.addAll(dir.getAbortedDirectories());
+    if (isDynPartAbort) {
+      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
+      // Including obsolete directories for partitioned tables can result in data loss.
+      obsoleteDirs = dir.getAbortedDirectories();
+    }
+    return obsoleteDirs;
+  }

Review Comment:
   Moved `getObsoleteDirs` to CompactorUtil since its using AcidDirectory which might not be generic to all handlers. However, the caching is done on a handler level now. Hence, caching functions are kept in handlers itself.
   `resolvePartition` seems generic to most handlers, hence kept it 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +47,31 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<CleaningRequestHandler> cleaningRequestHandlers;
+  private FSRemover fsRemover;
+
+  public Cleaner() {
+  }
+
+  public Cleaner(List<CleaningRequestHandler> cleaningRequestHandlers) {
+    this.cleaningRequestHandlers = cleaningRequestHandlers;
+  }
 
   @Override
   public void init(AtomicBoolean stop) throws Exception {
     super.init(stop);
-    replChangeManager = ReplChangeManager.getInstance(conf);
     checkInterval = conf.getTimeVar(
             HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, TimeUnit.MILLISECONDS);
     cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
             conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_THREADS_NUM),
             COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
     metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
         MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
-    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,
-            MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
-    initializeCache(tableCacheOn);
+    if (cleaningRequestHandlers == null || cleaningRequestHandlers.isEmpty()) {

Review Comment:
   could be replaced with CollectionUtils.isEmpty(cleaningRequestHandlers)



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {

Review Comment:
   do we expect any other cleanup request types? maybe `dropPartition` could have it's own type?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private final String runAs;
+  private final String cleanerMetric;
+  private final String dbName;
+  private final String tableName;
+  private final String partitionName;
+  private final boolean dropPartition;
+  private final String fullPartitionName;
+
+  public CleaningRequest(CleaningRequestBuilder<? extends CleaningRequestBuilder<?>> builder) {
+    this.type = builder.type;
+    this.location = builder.location;
+    this.obsoleteDirs = builder.obsoleteDirs;
+    this.purge = builder.purge;
+    this.fs = builder.fs;
+    this.runAs = builder.runAs;
+    this.cleanerMetric = builder.cleanerMetric;
+    this.dbName = builder.dbName;

Review Comment:
   can't we reuse the CI object?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private final String runAs;
+  private final String cleanerMetric;

Review Comment:
   isn't this a constant 
   ````
   MetricsConstants.COMPACTION_CLEANER_CYCLE + "_"
   ````



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionCleaningRequest.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+
+import java.util.Map;
+
+/**
+ * A cleaning request class specific to compaction based cleanup.
+ */
+public class CompactionCleaningRequest extends CleaningRequest {
+
+  private final CompactionInfo compactionInfo;
+  private final Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots;
+
+  public CompactionCleaningRequest(CompactionCleaningRequestBuilder builder) {

Review Comment:
   it's using the builder to construct itself? 



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CleaningRequestHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Map<CleaningRequest.RequestType, CleaningRequestHandler> handlerMap;

Review Comment:
   it doesn't need to know about the handlers, its job is to execute fs remove requests. use callback or something in clean method or refactor.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +47,31 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<CleaningRequestHandler> cleaningRequestHandlers;
+  private FSRemover fsRemover;
+
+  public Cleaner() {
+  }
+
+  public Cleaner(List<CleaningRequestHandler> cleaningRequestHandlers) {

Review Comment:
   use a setter instead of a constructor in the tests



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;

Review Comment:
   why does it have to be part of the request object?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -122,11 +80,8 @@ public void run() {
     try {
       do {
         TxnStore.MutexAPI.LockHandle handle = null;
-        invalidateMetaCache();
+        cleaningRequestHandlers.forEach(CleaningRequestHandler::invalidateMetaCache);

Review Comment:
   can't we reuse the cache between the handlers?



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CleaningRequestHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Map<CleaningRequest.RequestType, CleaningRequestHandler> handlerMap;

Review Comment:
   Removed the handlerMap and passed the handler object in clean() method itself.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [30 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [29 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [25 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [38 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [25 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandlerFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A factory class to fetch handlers.
+ */
+public class RequestHandlerFactory {

Review Comment:
   maybe `CleanerTaskFactory`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java:
##########
@@ -329,9 +335,9 @@ public void init(AtomicBoolean stop) throws Exception {
     compactionExecutor = CompactorUtil.createExecutorWithThreadFactory(
             conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_REQUEST_QUEUE),
             COMPACTOR_INTIATOR_THREAD_NAME_FORMAT);
-    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,
-        MetastoreConf.ConfVars.COMPACTOR_INITIATOR_TABLECACHE_ON);
-    initializeCache(tableCacheOn);
+//    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,

Review Comment:
   what are these comments? can this be removed



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

Review Comment:
   simply `get`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleanHandler.java:
##########
@@ -0,0 +1,391 @@
+/*
+ * 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.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+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.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleanupRequest;
+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.CompactorUtil.ThrowingRunnable;
+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.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+import static java.util.Objects.isNull;
+
+/**
+ * A compaction based implementation of RequestHandler.
+ * Provides implementation of creation of compaction clean tasks.
+ */
+class CompactionCleanHandler extends RequestHandler {

Review Comment:
   maybe `CompactionCleaner`, for aborted txns we could have `TxnAbortedCleaner`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java:
##########
@@ -97,16 +90,11 @@ public void init(AtomicBoolean stop) throws Exception {
   }
 
   @Override List<Partition> getPartitionsByNames(CompactionInfo ci) throws MetaException {
-    try {
-      return getMSForConf(conf).getPartitionsByNames(getDefaultCatalog(conf), ci.dbname, ci.tableName,
-          Collections.singletonList(ci.partName));
-    } catch (MetaException e) {
-      LOG.error("Unable to get partitions by name for CompactionInfo=" + ci);
-      throw e;
-    } catch (NoSuchObjectException e) {
-      LOG.error("Unable to get partitions by name for CompactionInfo=" + ci);
-      throw new MetaException(e.toString());
-    }
+    return CompactorUtil.getPartitionsByNames(conf, ci.dbname, ci.tableName, ci.partName);
+  }
+
+  public boolean isCacheEnabled() {

Review Comment:
   Done



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache;

Review Comment:
   Removed Optional. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Handler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  private Optional<Cache<String, TBase>> metaCache;
+
+  Handler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    boolean tableCacheOn = MetastoreConf.getBoolVar(this.conf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
+    this.metaCache = initializeCache(tableCacheOn);
+    this.metricsEnabled = metricsEnabled;
+  }
+
+  public HiveConf getConf() {
+    return conf;
+  }
+
+  public TxnStore getTxnHandler() {
+    return txnHandler;
+  }
+
+  public boolean isMetricsEnabled() {
+    return metricsEnabled;
+  }
+
+  /**
+   * Find the list of objects which are ready for cleaning.
+   * @return Cleaning requests
+   */
+  public abstract List<CleaningRequest> findReadyToClean() throws MetaException;
+
+  /**
+   * Execute just before cleanup
+   * @param cleaningRequest - Cleaning request
+   */
+  public abstract void beforeExecutingCleaningRequest(CleaningRequest cleaningRequest) throws MetaException;
+
+  /**
+   * Execute just after cleanup
+   * @param cleaningRequest Cleaning request
+   * @param deletedFiles List of deleted files
+   * @throws MetaException
+   */
+  public abstract void afterExecutingCleaningRequest(CleaningRequest cleaningRequest, List<Path> deletedFiles) throws MetaException;
+
+  /**
+   * Execute in the event of failure
+   * @param cleaningRequest Cleaning request
+   * @param ex Failure exception
+   * @throws MetaException
+   */
+  public abstract void failureExecutingCleaningRequest(CleaningRequest cleaningRequest, Exception ex) throws MetaException;

Review Comment:
   Insteadof this fixed method, you could implement an observer pattern: provide a method for registering exception handlers which will be executed in case of an exception. 
   
   That would give better flexibility: 
   1) The error handler can be completely independent of the Cleaning request handler (for example common handler for all handler implementations).
   2) It would be possible to register multiple errorhandlers, for example Cleaner could register its own as well.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionCleaningRequest.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A cleaning request class specific to compaction based cleanup.
+ */
+public class CompactionCleaningRequest extends CleaningRequest {
+
+  private final CompactionInfo compactionInfo;
+  private final Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots;
+
+  public CompactionCleaningRequest(String location, CompactionInfo info, List<Path> obsoleteDirs,
+                                   boolean purge, FileSystem fs, Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots,
+                                   boolean dropPartition) {
+    super(RequestType.COMPACTION, location, obsoleteDirs, purge, fs);
+    this.compactionInfo = info;
+    this.dbName = compactionInfo.dbname;
+    this.tableName = compactionInfo.tableName;
+    this.partitionName = compactionInfo.partName;
+    this.dirSnapshots = dirSnapshots;
+    this.cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
+            (compactionInfo.type != null ? compactionInfo.type.toString().toLowerCase() : null);
+    this.runAs = compactionInfo.runAs;
+    this.dropPartition = dropPartition;
+    this.fullPartitionName = compactionInfo.getFullPartitionName();
+  }
+
+  public CompactionInfo getCompactionInfo() {
+    return compactionInfo;
+  }
+
+  public Map<Path, AcidUtils.HdfsDirSnapshot> getHdfsDirSnapshots() {
+    return dirSnapshots;
+  }
+
+  @Override
+  public String toString() {

Review Comment:
   You could use org.apache.commons.lang3.builder.ToStringBuilder.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -212,323 +140,8 @@ public void run() {
     }
   }
 
-  private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
-    LOG.info("Starting cleaning for " + ci);
-    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
-    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
-        (ci.type != null ? ci.type.toString().toLowerCase() : null);
-    try {
-      if (metricsEnabled) {
-        perfLogger.perfLogBegin(CLASS_NAME, cleanerMetric);
-      }
-      final String location = ci.getProperty("location");
-
-      Callable<Boolean> cleanUpTask;
-      Table t = null;
-      Partition p = null;
-
-      if (location == null) {
-        t = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-        if (t == null) {
-          // The table was dropped before we got around to cleaning it.
-          LOG.info("Unable to find table " + ci.getFullTableName() + ", assuming it was dropped." +
-            idWatermark(ci));
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
-          // The table was marked no clean up true.
-          LOG.info("Skipping table " + ci.getFullTableName() + " clean up, as NO_CLEANUP set to true");
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (ci.partName != null) {
-          p = resolvePartition(ci);
-          if (p == null) {
-            // The partition was dropped before we got around to cleaning it.
-            LOG.info("Unable to find partition " + ci.getFullPartitionName() +
-              ", assuming it was dropped." + idWatermark(ci));
-            txnHandler.markCleaned(ci);
-            return;
-          }
-          if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
-            // The partition was marked no clean up true.
-            LOG.info("Skipping partition " + ci.getFullPartitionName() + " clean up, as NO_CLEANUP set to true");
-            txnHandler.markCleaned(ci);
-            return;
-          }
-        }
-      }
-      txnHandler.markCleanerStart(ci);
-
-      if (t != null || ci.partName != null) {
-        String path = location == null
-            ? resolveStorageDescriptor(t, p).getLocation()
-            : location;
-        boolean dropPartition = ci.partName != null && p == null;
-        cleanUpTask = () -> removeFiles(path, minOpenTxnGLB, ci, dropPartition);
-      } else {
-        cleanUpTask = () -> removeFiles(location, ci);
-      }
-
-      Ref<Boolean> removedFiles = Ref.from(false);
-      if (runJobAsSelf(ci.runAs)) {
-        removedFiles.value = cleanUpTask.call();
-      } else {
-        LOG.info("Cleaning as user " + ci.runAs + " for " + ci.getFullPartitionName());
-        UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs,
-            UserGroupInformation.getLoginUser());
-        try {
-          ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-            removedFiles.value = cleanUpTask.call();
-            return null;
-          });
-        } finally {
-          try {
-            FileSystem.closeAllForUGI(ugi);
-          } catch (IOException exception) {
-            LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " +
-                ci.getFullPartitionName() + idWatermark(ci), exception);
-          }
-        }
-      }
-      if (removedFiles.value || isDynPartAbort(t, ci)) {
-        txnHandler.markCleaned(ci);
-      } else {
-        txnHandler.clearCleanerStart(ci);
-        LOG.warn("No files were removed. Leaving queue entry " + ci + " in ready for cleaning state.");
-      }
-    } catch (Exception e) {
-      LOG.error("Caught exception when cleaning, unable to complete cleaning of " + ci + " " +
-          StringUtils.stringifyException(e));
-      ci.errorMessage = e.getMessage();
-      if (metricsEnabled) {
-        Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
-      }
-      handleCleanerAttemptFailure(ci);
-    }  finally {
-      if (metricsEnabled) {
-        perfLogger.perfLogEnd(CLASS_NAME, cleanerMetric);
-      }
-    }
-  }
-
-  private void handleCleanerAttemptFailure(CompactionInfo ci) throws MetaException {
-    long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS);
-    int cleanAttempts = 0;
-    if (ci.retryRetention > 0) {
-      cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1;
-    }
-    if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) {
-      //Mark it as failed if the max attempt threshold is reached.
-      txnHandler.markFailed(ci);
-    } else {
-      //Calculate retry retention time and update record.
-      ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention;
-      txnHandler.setCleanerRetryRetentionTimeOnError(ci);
-    }
-  }
-
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)
-      throws NoSuchTxnException, MetaException {
-    List<String> tblNames = Collections.singletonList(AcidUtils.getFullTableName(ci.dbname, ci.tableName));
-    GetValidWriteIdsRequest request = new GetValidWriteIdsRequest(tblNames);
-    request.setValidTxnList(validTxnList.writeToString());
-    GetValidWriteIdsResponse rsp = txnHandler.getValidWriteIds(request);
-    // we could have no write IDs for a table if it was never written to but
-    // since we are in the Cleaner phase of compactions, there must have
-    // been some delta/base dirs
-    assert rsp != null && rsp.getTblValidWriteIdsSize() == 1;
-    ValidReaderWriteIdList validWriteIdList =
-        TxnCommonUtils.createValidReaderWriteIdList(rsp.getTblValidWriteIds().get(0));
-    /*
-     * We need to filter the obsoletes dir list, to only remove directories that were made obsolete by this compaction
-     * If we have a higher retentionTime it is possible for a second compaction to run on the same partition. Cleaning up the first compaction
-     * should not touch the newer obsolete directories to not to violate the retentionTime for those.
-     */
-    if (ci.highestWriteId < validWriteIdList.getHighWatermark()) {
-      validWriteIdList = validWriteIdList.updateHighWatermark(ci.highestWriteId);
-    }
-    return validWriteIdList;
-  }
-
-  private static boolean isDynPartAbort(Table t, CompactionInfo ci) {
-    return Optional.ofNullable(t).map(Table::getPartitionKeys).filter(pk -> pk.size() > 0).isPresent()
-        && ci.partName == null;
-  }
-
-  private static String idWatermark(CompactionInfo ci) {
-    return " id=" + ci.id;
-  }
-
-  private boolean removeFiles(String location, long minOpenTxnGLB, CompactionInfo ci, boolean dropPartition)
-      throws Exception {
-
-    if (dropPartition) {
-      LockRequest lockRequest = createLockRequest(ci, 0, LockType.EXCL_WRITE, DataOperationType.DELETE);
-      LockResponse res = null;
-
-      try {
-        res = txnHandler.lock(lockRequest);
-        if (res.getState() == LockState.ACQUIRED) {
-          //check if partition wasn't re-created
-          if (resolvePartition(ci) == null) {
-            return removeFiles(location, ci);
-          }
-        }
-      } catch (NoSuchTxnException | TxnAbortedException e) {
-        LOG.error(e.getMessage());
-      } finally {
-        if (res != null) {
-          try {
-            txnHandler.unlock(new UnlockRequest(res.getLockid()));
-          } catch (NoSuchLockException | TxnOpenException e) {
-            LOG.error(e.getMessage());
-          }
-        }
-      }
-    }
-
-    ValidTxnList validTxnList =
-      TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB);
-    //save it so that getAcidState() sees it
-    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
-    /**
-     * {@code validTxnList} is capped by minOpenTxnGLB so if
-     * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
-     * produced by a compactor, that means every reader that could be active right now see it
-     * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
-     * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
-     *
-     *
-     * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
-     * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
-     * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
-     * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
-     * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
-     * writeId:17.  Compactor will produce base_17_c160.
-     * Suppose txnid:149 writes delta_18_18
-     * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
-     * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
-     * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
-     * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
-     * metadata that says that 18 is aborted.
-     * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
-     * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
-     * it (since it has nothing but aborted data).  But we can't tell which actually happened
-     * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
-     *
-     * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
-     * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
-     * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
-     * useful if there is all of a sudden a flood of aborted txns.  (For another day).
-     */
-
-    // 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);
-
-    return removeFiles(location, validWriteIdList, ci);
-  }
-  /**
-   * @return true if any files were removed
-   */
-  private boolean removeFiles(String location, ValidWriteIdList writeIdList, CompactionInfo ci)
-      throws Exception {
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-    
-    // Collect all of the files/dirs
-    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, path);
-    AcidDirectory dir = AcidUtils.getAcidState(fs, path, conf, writeIdList, Ref.from(false), false, 
-        dirSnapshots);
-    Table table = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-    boolean isDynPartAbort = isDynPartAbort(table, ci);
-    
-    List<Path> obsoleteDirs = getObsoleteDirs(dir, isDynPartAbort);
-    if (isDynPartAbort || dir.hasUncompactedAborts()) {
-      ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
-    }
-    List<Path> deleted = remove(location, ci, obsoleteDirs, true, fs);
-    if (dir.getObsolete().size() > 0) {
-      AcidMetricService.updateMetricsFromCleaner(ci.dbname, ci.tableName, ci.partName, dir.getObsolete(), conf,
-          txnHandler);
-    }
-    // Make sure there are no leftovers below the compacted watermark
-    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);
-    
-    List<Path> remained = subtract(getObsoleteDirs(dir, isDynPartAbort), deleted);
-    if (!remained.isEmpty()) {
-      LOG.warn(idWatermark(ci) + " Remained " + remained.size() +
-        " obsolete directories from " + location + ". " + getDebugInfo(remained));
-      return false;
-    }
-    LOG.debug(idWatermark(ci) + " All cleared below the watermark: " + ci.highestWriteId + " from " + location);
-    return true;
-  }
-  
-  private List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
-    List<Path> obsoleteDirs = dir.getObsolete();
-    /**
-     * add anything in 'dir'  that only has data from aborted transactions - no one should be
-     * trying to read anything in that dir (except getAcidState() that only reads the name of
-     * this dir itself)
-     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
-     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
-     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
-     * See {@link TxnStore#markCleaned(CompactionInfo)}
-     */
-    obsoleteDirs.addAll(dir.getAbortedDirectories());
-    if (isDynPartAbort) {
-      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
-      // Including obsolete directories for partitioned tables can result in data loss.
-      obsoleteDirs = dir.getAbortedDirectories();
-    }
-    return obsoleteDirs;
-  }
-
-  private boolean removeFiles(String location, CompactionInfo ci) throws IOException, MetaException {
-    String strIfPurge = ci.getProperty("ifPurge");
-    boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
-    
-    Path path = new Path(location);
-    return !remove(location, ci, Collections.singletonList(path), ifPurge,
-      path.getFileSystem(conf)).isEmpty();
-  }
-
-  private List<Path> remove(String location, CompactionInfo ci, List<Path> paths, boolean ifPurge, FileSystem fs)
-      throws MetaException, IOException {
-    List<Path> deleted = new ArrayList<>();
-    if (paths.size() < 1) {
-      return deleted;
-    }
-    LOG.info(idWatermark(ci) + " About to remove " + paths.size() +
-      " obsolete directories from " + location + ". " + getDebugInfo(paths));
-    boolean needCmRecycle;
-    try {
-      Database db = getMSForConf(conf).getDatabase(getDefaultCatalog(conf), ci.dbname);
-      needCmRecycle = ReplChangeManager.isSourceOfReplication(db);
-    } catch (NoSuchObjectException ex) {
-      // can not drop a database which is a source of replication
-      needCmRecycle = false;
-    }
-    for (Path dead : paths) {
-      LOG.debug("Going to delete path " + dead.toString());
-      if (needCmRecycle) {
-        replChangeManager.recycle(dead, ReplChangeManager.RecycleType.MOVE, ifPurge);
-      }
-      if (FileUtils.moveToTrash(fs, dead, conf, ifPurge)) {
-        deleted.add(dead);
-      }
-    }
-    return deleted;
-  }
-  
-  private String getDebugInfo(List<Path> paths) {
-    return "[" + paths.stream().map(Path::getName).collect(Collectors.joining(",")) + ']';
+  public void setHandlers(List<Handler> handlers) {

Review Comment:
   Instead of introducing this method for tests, you should have two constuctors and a HandlerFactory instance field like in Worker for CompactorFactory 



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler {

Review Comment:
   What about introducing a generic type here like Handler\<T extends CleaningRequest\> and using T in the methods like public abstract List\<T\> findReadyToClean() throws MetaException;?
   
   As a result CleaningRequest could contain only the fields necessary for the common part in Handler and FSRemover. Everything else could be moved completely to the descendant classes (if not moved yet) and the Handler implementations could be paired with them like: CompactionHandler extends Handler\<CleaningRequest\>. This would also eliminate the need for casting in CompactionHandler. 



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -141,45 +85,29 @@ public void run() {
                     new CleanerCycleUpdater(MetricsConstants.COMPACTION_CLEANER_CYCLE_DURATION, startedAt));
           }
 
-          long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
-
-          checkInterrupt();
-
-          List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
+          for (Handler handler : handlers) {
+            List<CleaningRequest> readyToClean = handler.findReadyToClean();

Review Comment:
   If the frst handler is broken for whatever reason, the remaining handlers won't get any chance to provide cleaning requests. This method call should be handled in a way to continue the handler iteration so subsquent ones can stil provide CleaningRequests



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HandlerFactory.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CompactionHandler;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.Handler;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A factory class to fetch handlers.
+ */
+public class HandlerFactory {

Review Comment:
   This could be moved inside handler package, so the concrete Handler implementations could be package private in order to prevent client code to obtain them anyhow else than through the factory



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  protected String runAs;

Review Comment:
   If you want to restrict access for these fields, these should be either private or the class should be moved into another package. With the protected access right now they are wisible everywhere in the org.apache.hadoop.hive.ql.txn.compactor package which I guess not the desired visibility level.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionHandler.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+
+/**
+ * A compaction based implementation of Handler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+public class CompactionHandler extends Handler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionHandler.class.getName());
+
+  public CompactionHandler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    super(conf, txnHandler, metricsEnabled);
+  }
+
+  @Override
+  public List<CleaningRequest> findReadyToClean() throws MetaException {
+    List<CleaningRequest> cleaningRequests = new ArrayList<>();
+    long retentionTime = HiveConf.getBoolVar(conf, HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED)
+            ? HiveConf.getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETENTION_TIME, TimeUnit.MILLISECONDS)
+            : 0;
+    long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
+    List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
+
+    if (!readyToClean.isEmpty()) {
+      long minTxnIdSeenOpen = txnHandler.findMinTxnIdSeenOpen();
+      final long cleanerWaterMark =
+              minTxnIdSeenOpen < 0 ? minOpenTxnId : Math.min(minOpenTxnId, minTxnIdSeenOpen);
+
+      LOG.info("Cleaning based on min open txn id: {}", cleanerWaterMark);
+      // For checking which compaction can be cleaned we can use the minOpenTxnId
+      // However findReadyToClean will return all records that were compacted with old version of HMS
+      // where the CQ_NEXT_TXN_ID is not set. For these compactions we need to provide minTxnIdSeenOpen
+      // to the clean method, to avoid cleaning up deltas needed for running queries
+      // when min_history_level is finally dropped, than every HMS will commit compaction the new way
+      // and minTxnIdSeenOpen can be removed and minOpenTxnId can be used instead.
+      for (CompactionInfo ci : readyToClean) {
+        LOG.info("Starting cleaning for {}", ci);
+        try {
+          final String location = ci.getProperty("location");
+
+          Table t = null;
+          Partition p = null;
+
+          if (location == null) {
+            t = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.tableName));
+            if (t == null) {
+              // The table was dropped before we got around to cleaning it.
+              LOG.info("Unable to find table {}, assuming it was dropped. {}", ci.getFullTableName(),
+                      idWatermark(ci));
+              txnHandler.markCleaned(ci);
+              continue;
+            }
+            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", ci.getFullTableName());
+              txnHandler.markCleaned(ci);
+              continue;
+            }
+            if (ci.partName != null) {
+              p = resolvePartition(ci.dbname, ci.tableName, ci.partName);
+              if (p == null) {
+                // The partition was dropped before we got around to cleaning it.
+                LOG.info("Unable to find partition {}, assuming it was dropped. {}", ci.getFullTableName(), idWatermark(ci));
+                txnHandler.markCleaned(ci);
+                continue;
+              }
+              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", ci.getFullPartitionName());
+                txnHandler.markCleaned(ci);
+                continue;
+              }
+            }
+          }
+
+          if (t != null || ci.partName != null) {
+            String path = location == null
+                    ? CompactorUtil.resolveStorageDescriptor(t, p).getLocation()
+                    : location;
+            boolean dropPartition = ci.partName != null && p == null;
+            if (dropPartition) {
+              //check if partition wasn't re-created
+              if (resolvePartition(ci.dbname, ci.tableName, ci.partName) == null) {
+                String strIfPurge = ci.getProperty("ifPurge");
+                boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
+
+                Path obsoletePath = new Path(path);
+                cleaningRequests.add(new CompactionCleaningRequest(path, ci, Collections.singletonList(obsoletePath), ifPurge,
+                        obsoletePath.getFileSystem(conf), null, true));
+                continue;
+              }
+            }
+
+            ValidTxnList validTxnList =
+                    TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), cleanerWaterMark);
+            //save it so that getAcidState() sees it
+            conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+            /*
+             * {@code validTxnList} is capped by minOpenTxnGLB so if
+             * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
+             * produced by a compactor, that means every reader that could be active right now see it
+             * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
+             * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
+             *
+             *
+             * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
+             * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
+             * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
+             * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
+             * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
+             * writeId:17.  Compactor will produce base_17_c160.
+             * Suppose txnid:149 writes delta_18_18
+             * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
+             * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
+             * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
+             * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
+             * metadata that says that 18 is aborted.
+             * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
+             * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
+             * it (since it has nothing but aborted data).  But we can't tell which actually happened
+             * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
+             *
+             * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
+             * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
+             * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
+             * useful if there is all of a sudden a flood of aborted txns.  (For another day).
+             */
+
+            // 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 loc = new Path(path);
+            FileSystem fs = loc.getFileSystem(conf);
+
+            // Collect all the files/dirs
+            Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, loc);
+            AcidDirectory dir = AcidUtils.getAcidState(fs, loc, conf, validWriteIdList, Ref.from(false), false,
+                    dirSnapshots);
+            Table table = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.tableName));
+            boolean isDynPartAbort = CompactorUtil.isDynPartAbort(table, ci.partName);
+
+            List<Path> obsoleteDirs = getObsoleteDirs(dir, isDynPartAbort);
+            if (isDynPartAbort || dir.hasUncompactedAborts()) {
+              ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
+            }
+
+            cleaningRequests.add(new CompactionCleaningRequest(path, ci, obsoleteDirs, true, fs, dirSnapshots, false));
+          } else {
+            String strIfPurge = ci.getProperty("ifPurge");
+            boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
+
+            Path obsoletePath = new Path(location);
+            cleaningRequests.add(new CompactionCleaningRequest(location, ci, Collections.singletonList(obsoletePath), ifPurge,
+                    obsoletePath.getFileSystem(conf), null, false));
+          }
+        } catch (Exception e) {
+          LOG.warn("Cleaning request was not successful generated for : {} due to {}", idWatermark(ci), e.getMessage());
+          ci.errorMessage = e.getMessage();
+          if (metricsEnabled) {
+            Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
+          }
+          handleCleanerAttemptFailure(ci);
+        }
+      }
+    }
+    return cleaningRequests;
+  }
+
+  @Override
+  public void beforeExecutingCleaningRequest(CleaningRequest cleaningRequest) throws MetaException {
+    CompactionInfo ci = ((CompactionCleaningRequest) cleaningRequest).getCompactionInfo();
+    txnHandler.markCleanerStart(ci);
+  }
+
+  @Override
+  public void afterExecutingCleaningRequest(CleaningRequest cleaningRequest, List<Path> deletedFiles) throws MetaException {
+    CompactionInfo ci = ((CompactionCleaningRequest) cleaningRequest).getCompactionInfo();
+    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = ((CompactionCleaningRequest) cleaningRequest).getHdfsDirSnapshots();
+    // Make sure there are no leftovers below the compacted watermark
+    if (dirSnapshots != null) {
+      conf.set(ValidTxnList.VALID_TXNS_KEY, new ValidReadTxnList().toString());
+      Path path = new Path(cleaningRequest.getLocation());
+      Table table;
+      boolean success = false;

Review Comment:
   Is it possible that FSRemover shares some information about the result of the file removal? Ideally the FileSystem related tasks should be extracted from the handlers



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  protected final MetadataCache metadataCache;
+  protected final FSRemover fsRemover;
+  protected final ExecutorService cleanerExecutor;
+
+  RequestHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
+                         boolean metricsEnabled, FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    this.metadataCache = metadataCache;
+    this.metricsEnabled = metricsEnabled;
+    this.fsRemover = fsRemover;
+    this.cleanerExecutor = cleanerExecutor;
+  }
+
+  protected abstract List<Runnable> fetchCleanTasks() throws MetaException;
+
+  public void process() throws Exception {

Review Comment:
   process() should accept the executionPool



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

Review Comment:
   What is the usage pattern for this, do we use the return type anywhere? 
   should it be simple `public void init(boolean enabled)`?



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

To unsubscribe, e-mail: 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects of this class are created by request handlers.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {

Review Comment:
   Renamed. Done.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -212,323 +138,9 @@ public void run() {
     }
   }
 
-  private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
-    LOG.info("Starting cleaning for " + ci);
-    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
-    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
-        (ci.type != null ? ci.type.toString().toLowerCase() : null);
-    try {
-      if (metricsEnabled) {
-        perfLogger.perfLogBegin(CLASS_NAME, cleanerMetric);
-      }
-      final String location = ci.getProperty("location");
-
-      Callable<Boolean> cleanUpTask;
-      Table t = null;
-      Partition p = null;
-
-      if (location == null) {
-        t = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-        if (t == null) {
-          // The table was dropped before we got around to cleaning it.
-          LOG.info("Unable to find table " + ci.getFullTableName() + ", assuming it was dropped." +
-            idWatermark(ci));
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
-          // The table was marked no clean up true.
-          LOG.info("Skipping table " + ci.getFullTableName() + " clean up, as NO_CLEANUP set to true");
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (ci.partName != null) {
-          p = resolvePartition(ci);
-          if (p == null) {
-            // The partition was dropped before we got around to cleaning it.
-            LOG.info("Unable to find partition " + ci.getFullPartitionName() +
-              ", assuming it was dropped." + idWatermark(ci));
-            txnHandler.markCleaned(ci);
-            return;
-          }
-          if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
-            // The partition was marked no clean up true.
-            LOG.info("Skipping partition " + ci.getFullPartitionName() + " clean up, as NO_CLEANUP set to true");
-            txnHandler.markCleaned(ci);
-            return;
-          }
-        }
-      }
-      txnHandler.markCleanerStart(ci);
-
-      if (t != null || ci.partName != null) {
-        String path = location == null
-            ? resolveStorageDescriptor(t, p).getLocation()
-            : location;
-        boolean dropPartition = ci.partName != null && p == null;
-        cleanUpTask = () -> removeFiles(path, minOpenTxnGLB, ci, dropPartition);
-      } else {
-        cleanUpTask = () -> removeFiles(location, ci);
-      }
-
-      Ref<Boolean> removedFiles = Ref.from(false);
-      if (runJobAsSelf(ci.runAs)) {
-        removedFiles.value = cleanUpTask.call();
-      } else {
-        LOG.info("Cleaning as user " + ci.runAs + " for " + ci.getFullPartitionName());
-        UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs,
-            UserGroupInformation.getLoginUser());
-        try {
-          ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-            removedFiles.value = cleanUpTask.call();
-            return null;
-          });
-        } finally {
-          try {
-            FileSystem.closeAllForUGI(ugi);
-          } catch (IOException exception) {
-            LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " +
-                ci.getFullPartitionName() + idWatermark(ci), exception);
-          }
-        }
-      }
-      if (removedFiles.value || isDynPartAbort(t, ci)) {
-        txnHandler.markCleaned(ci);
-      } else {
-        txnHandler.clearCleanerStart(ci);
-        LOG.warn("No files were removed. Leaving queue entry " + ci + " in ready for cleaning state.");
-      }
-    } catch (Exception e) {
-      LOG.error("Caught exception when cleaning, unable to complete cleaning of " + ci + " " +
-          StringUtils.stringifyException(e));
-      ci.errorMessage = e.getMessage();
-      if (metricsEnabled) {
-        Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
-      }
-      handleCleanerAttemptFailure(ci);
-    }  finally {
-      if (metricsEnabled) {
-        perfLogger.perfLogEnd(CLASS_NAME, cleanerMetric);
-      }
-    }
-  }
-
-  private void handleCleanerAttemptFailure(CompactionInfo ci) throws MetaException {
-    long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS);
-    int cleanAttempts = 0;
-    if (ci.retryRetention > 0) {
-      cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1;
-    }
-    if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) {
-      //Mark it as failed if the max attempt threshold is reached.
-      txnHandler.markFailed(ci);
-    } else {
-      //Calculate retry retention time and update record.
-      ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention;
-      txnHandler.setCleanerRetryRetentionTimeOnError(ci);
-    }
-  }
-
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)
-      throws NoSuchTxnException, MetaException {
-    List<String> tblNames = Collections.singletonList(AcidUtils.getFullTableName(ci.dbname, ci.tableName));
-    GetValidWriteIdsRequest request = new GetValidWriteIdsRequest(tblNames);
-    request.setValidTxnList(validTxnList.writeToString());
-    GetValidWriteIdsResponse rsp = txnHandler.getValidWriteIds(request);
-    // we could have no write IDs for a table if it was never written to but
-    // since we are in the Cleaner phase of compactions, there must have
-    // been some delta/base dirs
-    assert rsp != null && rsp.getTblValidWriteIdsSize() == 1;
-    ValidReaderWriteIdList validWriteIdList =
-        TxnCommonUtils.createValidReaderWriteIdList(rsp.getTblValidWriteIds().get(0));
-    /*
-     * We need to filter the obsoletes dir list, to only remove directories that were made obsolete by this compaction
-     * If we have a higher retentionTime it is possible for a second compaction to run on the same partition. Cleaning up the first compaction
-     * should not touch the newer obsolete directories to not to violate the retentionTime for those.
-     */
-    if (ci.highestWriteId < validWriteIdList.getHighWatermark()) {
-      validWriteIdList = validWriteIdList.updateHighWatermark(ci.highestWriteId);
-    }
-    return validWriteIdList;
-  }
-
-  private static boolean isDynPartAbort(Table t, CompactionInfo ci) {
-    return Optional.ofNullable(t).map(Table::getPartitionKeys).filter(pk -> pk.size() > 0).isPresent()
-        && ci.partName == null;
-  }
-
-  private static String idWatermark(CompactionInfo ci) {
-    return " id=" + ci.id;
-  }
-
-  private boolean removeFiles(String location, long minOpenTxnGLB, CompactionInfo ci, boolean dropPartition)
-      throws Exception {
-
-    if (dropPartition) {
-      LockRequest lockRequest = createLockRequest(ci, 0, LockType.EXCL_WRITE, DataOperationType.DELETE);
-      LockResponse res = null;
-
-      try {
-        res = txnHandler.lock(lockRequest);
-        if (res.getState() == LockState.ACQUIRED) {
-          //check if partition wasn't re-created
-          if (resolvePartition(ci) == null) {
-            return removeFiles(location, ci);
-          }
-        }
-      } catch (NoSuchTxnException | TxnAbortedException e) {
-        LOG.error(e.getMessage());
-      } finally {
-        if (res != null) {
-          try {
-            txnHandler.unlock(new UnlockRequest(res.getLockid()));
-          } catch (NoSuchLockException | TxnOpenException e) {
-            LOG.error(e.getMessage());
-          }
-        }
-      }
-    }
-
-    ValidTxnList validTxnList =
-      TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB);
-    //save it so that getAcidState() sees it
-    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
-    /**
-     * {@code validTxnList} is capped by minOpenTxnGLB so if
-     * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
-     * produced by a compactor, that means every reader that could be active right now see it
-     * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
-     * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
-     *
-     *
-     * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
-     * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
-     * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
-     * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
-     * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
-     * writeId:17.  Compactor will produce base_17_c160.
-     * Suppose txnid:149 writes delta_18_18
-     * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
-     * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
-     * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
-     * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
-     * metadata that says that 18 is aborted.
-     * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
-     * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
-     * it (since it has nothing but aborted data).  But we can't tell which actually happened
-     * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
-     *
-     * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
-     * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
-     * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
-     * useful if there is all of a sudden a flood of aborted txns.  (For another day).
-     */
-
-    // 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);
-
-    return removeFiles(location, validWriteIdList, ci);
-  }
-  /**
-   * @return true if any files were removed
-   */
-  private boolean removeFiles(String location, ValidWriteIdList writeIdList, CompactionInfo ci)
-      throws Exception {
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-    
-    // Collect all of the files/dirs
-    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, path);
-    AcidDirectory dir = AcidUtils.getAcidState(fs, path, conf, writeIdList, Ref.from(false), false, 
-        dirSnapshots);
-    Table table = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-    boolean isDynPartAbort = isDynPartAbort(table, ci);
-    
-    List<Path> obsoleteDirs = getObsoleteDirs(dir, isDynPartAbort);
-    if (isDynPartAbort || dir.hasUncompactedAborts()) {
-      ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
-    }
-    List<Path> deleted = remove(location, ci, obsoleteDirs, true, fs);
-    if (dir.getObsolete().size() > 0) {
-      AcidMetricService.updateMetricsFromCleaner(ci.dbname, ci.tableName, ci.partName, dir.getObsolete(), conf,
-          txnHandler);
-    }
-    // Make sure there are no leftovers below the compacted watermark
-    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);
-    
-    List<Path> remained = subtract(getObsoleteDirs(dir, isDynPartAbort), deleted);
-    if (!remained.isEmpty()) {
-      LOG.warn(idWatermark(ci) + " Remained " + remained.size() +
-        " obsolete directories from " + location + ". " + getDebugInfo(remained));
-      return false;
-    }
-    LOG.debug(idWatermark(ci) + " All cleared below the watermark: " + ci.highestWriteId + " from " + location);
-    return true;
-  }
-  
-  private List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
-    List<Path> obsoleteDirs = dir.getObsolete();
-    /**
-     * add anything in 'dir'  that only has data from aborted transactions - no one should be
-     * trying to read anything in that dir (except getAcidState() that only reads the name of
-     * this dir itself)
-     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
-     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
-     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
-     * See {@link TxnStore#markCleaned(CompactionInfo)}
-     */
-    obsoleteDirs.addAll(dir.getAbortedDirectories());
-    if (isDynPartAbort) {
-      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
-      // Including obsolete directories for partitioned tables can result in data loss.
-      obsoleteDirs = dir.getAbortedDirectories();
-    }
-    return obsoleteDirs;
-  }
-
-  private boolean removeFiles(String location, CompactionInfo ci) throws IOException, MetaException {
-    String strIfPurge = ci.getProperty("ifPurge");
-    boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
-    
-    Path path = new Path(location);
-    return !remove(location, ci, Collections.singletonList(path), ifPurge,
-      path.getFileSystem(conf)).isEmpty();
-  }
-
-  private List<Path> remove(String location, CompactionInfo ci, List<Path> paths, boolean ifPurge, FileSystem fs)
-      throws MetaException, IOException {
-    List<Path> deleted = new ArrayList<>();
-    if (paths.size() < 1) {
-      return deleted;
-    }
-    LOG.info(idWatermark(ci) + " About to remove " + paths.size() +
-      " obsolete directories from " + location + ". " + getDebugInfo(paths));
-    boolean needCmRecycle;
-    try {
-      Database db = getMSForConf(conf).getDatabase(getDefaultCatalog(conf), ci.dbname);
-      needCmRecycle = ReplChangeManager.isSourceOfReplication(db);
-    } catch (NoSuchObjectException ex) {
-      // can not drop a database which is a source of replication
-      needCmRecycle = false;
-    }
-    for (Path dead : paths) {
-      LOG.debug("Going to delete path " + dead.toString());
-      if (needCmRecycle) {
-        replChangeManager.recycle(dead, ReplChangeManager.RecycleType.MOVE, ifPurge);
-      }
-      if (FileUtils.moveToTrash(fs, dead, conf, ifPurge)) {
-        deleted.add(dead);
-      }
-    }
-    return deleted;
-  }
-  
-  private String getDebugInfo(List<Path> paths) {
-    return "[" + paths.stream().map(Path::getName).collect(Collectors.joining(",")) + ']';
+  @VisibleForTesting
+  public void setCleaningRequestHandlers(List<RequestHandler> requestHandlers) {

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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java:
##########
@@ -57,15 +50,15 @@ public class MetaStoreCompactorThread extends CompactorThread implements MetaSto
 
   protected TxnStore txnHandler;
   protected ScheduledExecutorService cycleUpdaterExecutorService;
-
-  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+  protected CacheContainer cacheContainer;
 
   @Override
   public void init(AtomicBoolean stop) throws Exception {
     super.init(stop);
 
     // Get our own instance of the transaction handler
     txnHandler = TxnUtils.getTxnStore(conf);
+    cacheContainer = new CacheContainer();

Review Comment:
   As discussed, the cached instances are invalidated both on Initiator and Cleaner side. There might also be staleness of table object if properties of the table are changed in the time between Initiator & Cleaner.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandlerFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A factory class to fetch handlers.
+ */
+public class RequestHandlerFactory {
+  private static final RequestHandlerFactory INSTANCE = new RequestHandlerFactory();
+
+  public static RequestHandlerFactory getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Factory class, no need to expose constructor.
+   */
+  private RequestHandlerFactory() {
+  }
+
+  public List<RequestHandler> getHandlers(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,

Review Comment:
   `getTasks`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [31 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  protected final MetadataCache metadataCache;
+  protected final FSRemover fsRemover;
+  protected final ExecutorService cleanerExecutor;
+
+  RequestHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
+                         boolean metricsEnabled, FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    this.metadataCache = metadataCache;
+    this.metricsEnabled = metricsEnabled;
+    this.fsRemover = fsRemover;
+    this.cleanerExecutor = cleanerExecutor;
+  }
+
+  protected abstract List<Runnable> fetchCleanTasks() throws MetaException;
+
+  public void process() throws Exception {

Review Comment:
   As discussed fsRemover is required. However, cleaner gets the tasks and it executes the tasks using the executor pool.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  protected final MetadataCache metadataCache;
+  protected final FSRemover fsRemover;
+  protected final ExecutorService cleanerExecutor;
+
+  RequestHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
+                         boolean metricsEnabled, FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    this.metadataCache = metadataCache;
+    this.metricsEnabled = metricsEnabled;
+    this.fsRemover = fsRemover;
+    this.cleanerExecutor = cleanerExecutor;
+  }
+
+  protected abstract List<Runnable> fetchCleanTasks() throws MetaException;

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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaningRequestHandler.java:
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest.CompactionCleaningRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+
+/**
+ * A compaction based implementation of CleaningRequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionCleaningRequestHandler extends CleaningRequestHandler<CompactionCleaningRequest> {

Review Comment:
   @deniskuzZ I understand the naming is too big. However, the suggested one might cause confusion since we already have a "Cleaner" thread. 
   I would suggest the following - 
   1. RequestHandler - The generic class for generating the request objects.
      a. CompactionHandler - The class which generates compaction request.
   2. CleaningRequest - The generic class for cleaning requests.
      a. CompactionRequest - The class which has compaction related information.
      
   WDYT?



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

To unsubscribe, e-mail: 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CleaningRequestHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Map<CleaningRequest.RequestType, CleaningRequestHandler> handlerMap;

Review Comment:
   FSRemover is no more dependent on Handler on any methods after refactor. All invocations are made by handler to the FSRemover 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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CleaningRequestHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Map<CleaningRequest.RequestType, CleaningRequestHandler> handlerMap;

Review Comment:
   Removed the handlerMap and passed the handler object in clean() method itself.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionCleaningRequest.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+
+import java.util.Map;
+
+/**
+ * A cleaning request class specific to compaction based cleanup.
+ */
+public class CompactionCleaningRequest extends CleaningRequest {
+
+  private final CompactionInfo compactionInfo;
+  private final Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots;
+
+  public CompactionCleaningRequest(CompactionCleaningRequestBuilder builder) {

Review Comment:
   This file is removed after refactor. So marking this as resolved.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java:
##########
@@ -57,15 +50,15 @@ public class MetaStoreCompactorThread extends CompactorThread implements MetaSto
 
   protected TxnStore txnHandler;
   protected ScheduledExecutorService cycleUpdaterExecutorService;
-
-  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+  protected CacheContainer cacheContainer;
 
   @Override
   public void init(AtomicBoolean stop) throws Exception {
     super.init(stop);
 
     // Get our own instance of the transaction handler
     txnHandler = TxnUtils.getTxnStore(conf);
+    cacheContainer = new CacheContainer();

Review Comment:
   shouldn't it be a singleton? why do we need multiple cache instances for Initiator & Cleaner? 



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [31 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [31 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [41 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler<T extends CleaningRequest> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Handler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  private Optional<Cache<String, TBase>> metaCache;
+
+  Handler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    boolean tableCacheOn = MetastoreConf.getBoolVar(this.conf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
+    this.metaCache = initializeCache(tableCacheOn);
+    this.metricsEnabled = metricsEnabled;
+  }
+
+  public HiveConf getConf() {
+    return conf;
+  }
+
+  public TxnStore getTxnHandler() {
+    return txnHandler;
+  }
+
+  public boolean isMetricsEnabled() {
+    return metricsEnabled;
+  }
+
+  /**
+   * Find the list of objects which are ready for cleaning.
+   * @return Cleaning requests
+   */
+  public abstract List<T> findReadyToClean() throws MetaException;
+
+  /**
+   * Execute just before cleanup
+   * @param cleaningRequest - Cleaning request
+   */
+  public abstract void beforeExecutingCleaningRequest(T cleaningRequest) throws MetaException;
+
+  /**
+   * Execute just after cleanup
+   * @param cleaningRequest Cleaning request
+   * @param deletedFiles List of deleted files
+   * @return True if cleanup was successful, false otherwise
+   * @throws MetaException
+   */
+  public abstract boolean afterExecutingCleaningRequest(T cleaningRequest, List<Path> deletedFiles) throws MetaException;
+
+  /**
+   * Execute in the event of failure
+   * @param cleaningRequest Cleaning request
+   * @param ex Failure exception
+   * @throws MetaException
+   */
+  public abstract void failureExecutingCleaningRequest(T cleaningRequest, Exception ex) throws MetaException;
+
+  public Table resolveTable(String dbName, String tableName) throws MetaException {
+    try {
+      return getMSForConf(conf).getTable(getDefaultCatalog(conf), dbName, tableName);
+    } catch (MetaException e) {
+      LOG.error("Unable to find table {}.{}, {}", dbName, tableName, e.getMessage());
+      throw e;
+    }
+  }
+
+  protected Partition resolvePartition(String dbName, String tableName, String partName) throws MetaException {
+    if (partName != null) {
+      List<Partition> parts;
+      try {
+        parts = CompactorUtil.getPartitionsByNames(conf, dbName, tableName, partName);
+        if (parts == null || parts.isEmpty()) {
+          // The partition got dropped before we went looking for it.
+          return null;
+        }
+      } catch (Exception e) {
+        LOG.error("Unable to find partition: {}.{}.{}", dbName, tableName, partName, e);
+        throw e;
+      }
+      if (parts.size() != 1) {
+        LOG.error("{}.{}.{} does not refer to a single partition. {}", dbName, tableName, partName,
+                Arrays.toString(parts.toArray()));
+        throw new MetaException(String.join("Too many partitions for : ", dbName, tableName, partName));
+      }
+      return parts.get(0);
+    } else {
+      return null;
+    }
+  }
+
+  public <B extends TBase<B,?>> B computeIfAbsent(String key, Callable<B> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (B) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {
+    if (tableCacheOn) {
+      metaCache = Optional.of(CacheBuilder.newBuilder().softValues().build());
+    }
+    return metaCache;
+  }
+
+  public void invalidateMetaCache() {
+    metaCache.ifPresent(Cache::invalidateAll);
+  }
+
+  protected List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
+    List<Path> obsoleteDirs = dir.getObsolete();
+    /*
+     * add anything in 'dir'  that only has data from aborted transactions - no one should be
+     * trying to read anything in that dir (except getAcidState() that only reads the name of
+     * this dir itself)
+     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
+     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
+     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
+     * See {@link TxnStore#markCleaned(CompactionInfo)}
+     */
+    obsoleteDirs.addAll(dir.getAbortedDirectories());
+    if (isDynPartAbort) {
+      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
+      // Including obsolete directories for partitioned tables can result in data loss.
+      obsoleteDirs = dir.getAbortedDirectories();
+    }
+    return obsoleteDirs;
+  }

Review Comment:
   Moved `getObsoleteDirs` to CompactorUtil since its using AcidDirectory which might not be generic to all handlers. However, the caching is done on a handler level now. Hence, caching functions are kept in handlers itself.
   `resolvePartitions` seems generic to most handlers, hence kept it 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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestHandler.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.Cleaner;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCleaner;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+
+public class TestHandler extends TestCleaner {
+
+  @Test
+  public void testCompactionHandlerForSuccessfulCompaction() throws Exception {
+    Table t = newTable("default", "handler_success_table", true);
+    Partition p = newPartition(t, "today");
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
+
+    burnThroughTransactions(t.getDbName(), t.getTableName(), 25);
+
+    CompactionRequest rqst = new CompactionRequest(t.getDbName(), t.getTableName(), CompactionType.MAJOR);
+    rqst.setPartitionname("ds=today");
+    compactInTxn(rqst);
+
+    Handler handler = new CompactionHandler(conf, txnHandler, false);
+
+    // Fetch the compaction request using the handler
+    List<CleaningRequest> cleaningRequests = handler.findReadyToClean();
+    Assert.assertEquals(1, cleaningRequests.size());
+    CleaningRequest cr = cleaningRequests.get(0);
+    Assert.assertEquals(t.getDbName(), cr.getDbName());
+    Assert.assertEquals(t.getTableName(), cr.getTableName());
+    Assert.assertEquals("ds=today", cr.getPartitionName());
+    Assert.assertEquals(CleaningRequest.RequestType.COMPACTION, cr.getType());
+
+    // Check whether appropriate handler utility methods are called exactly once in a successful compaction scenario.
+    Handler mockedHandler = Mockito.spy(handler);
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Cleaner cleaner = new Cleaner(Arrays.asList(mockedHandler));
+    cleaner.setConf(conf);
+    cleaner.init(stop);
+    cleaner.run();
+
+    Mockito.verify(mockedHandler, Mockito.times(1)).findReadyToClean();
+    Mockito.verify(mockedHandler, Mockito.times(1)).beforeExecutingCleaningRequest(any(CleaningRequest.class));
+    Mockito.verify(mockedHandler, Mockito.times(1)).afterExecutingCleaningRequest(any(CleaningRequest.class), any(List.class));
+  }
+
+  @Test
+  public void testCompactionHandlerForFailureCompaction() throws Exception {
+    Table t = newTable("default", "handler_failure_table", true);
+    Partition p = newPartition(t, "today");
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
+
+    burnThroughTransactions(t.getDbName(), t.getTableName(), 25);
+
+    CompactionRequest rqst = new CompactionRequest(t.getDbName(), t.getTableName(), CompactionType.MAJOR);
+    rqst.setPartitionname("ds=today");
+    compactInTxn(rqst);
+
+    // Check whether appropriate handler utility methods are called exactly once in a failure compaction scenario.
+    TxnStore mockedTxnHandler = Mockito.spy(txnHandler);
+    doThrow(new RuntimeException()).when(mockedTxnHandler).markCleaned(any());
+    Handler mockedHandler = Mockito.spy(new CompactionHandler(conf, mockedTxnHandler, false));
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Cleaner cleaner = new Cleaner(Arrays.asList(mockedHandler));
+    cleaner.setConf(conf);
+    cleaner.init(stop);
+    cleaner.run();
+
+    Mockito.verify(mockedHandler, Mockito.times(1)).findReadyToClean();
+    Mockito.verify(mockedHandler, Mockito.times(1)).beforeExecutingCleaningRequest(any(CleaningRequest.class));
+    Mockito.verify(mockedHandler, Mockito.times(1)).failureExecutingCleaningRequest(any(CleaningRequest.class), any(Exception.class));

Review Comment:
   Added assert. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -39,38 +49,44 @@
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Callable;
 
 import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 
 /**
- * A runnable class which takes in handler and cleaning request and deletes the files
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
  * according to the cleaning request.
  */
-public class FSRemover implements ThrowingRunnable<MetaException> {
+public class FSRemover {
   private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
-  private final Handler handler;
-  private final CleaningRequest cr;
+  private final Map<CleaningRequest.RequestType, CleaningRequestHandler> handlerMap;

Review Comment:
   I like this approach!



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -194,8 +210,17 @@ private LockRequest createLockRequest(String dbName, String tableName, String pa
     }

Review Comment:
   `protected LockRequest createLockRequest` is called only once, and it is the only method calling the `private LockRequest createLockRequest`, so there is no need for two overloads.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;

Review Comment:
   This is used in many places and I thought creating this might be expensive (not sure). Hence thought of keeping this in the cleaning request itself.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaningRequestHandler.java:
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactionCleaningRequest.CompactionCleaningRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+
+/**
+ * A compaction based implementation of CleaningRequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionCleaningRequestHandler extends CleaningRequestHandler<CompactionCleaningRequest> {

Review Comment:
   maybe simply 
   ````
   CompactionCleaner extends Cleaner<CleanupRequest>?
   ````



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +47,31 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<CleaningRequestHandler> cleaningRequestHandlers;
+  private FSRemover fsRemover;
+
+  public Cleaner() {
+  }
+
+  public Cleaner(List<CleaningRequestHandler> cleaningRequestHandlers) {

Review Comment:
   Done.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +47,31 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<CleaningRequestHandler> cleaningRequestHandlers;
+  private FSRemover fsRemover;
+
+  public Cleaner() {
+  }
+
+  public Cleaner(List<CleaningRequestHandler> cleaningRequestHandlers) {
+    this.cleaningRequestHandlers = cleaningRequestHandlers;
+  }
 
   @Override
   public void init(AtomicBoolean stop) throws Exception {
     super.init(stop);
-    replChangeManager = ReplChangeManager.getInstance(conf);
     checkInterval = conf.getTimeVar(
             HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, TimeUnit.MILLISECONDS);
     cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
             conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_THREADS_NUM),
             COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
     metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
         MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
-    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,
-            MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
-    initializeCache(tableCacheOn);
+    if (cleaningRequestHandlers == null || cleaningRequestHandlers.isEmpty()) {

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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CleaningRequestHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Map<CleaningRequest.RequestType, CleaningRequestHandler> handlerMap;

Review Comment:
   FSRemover is no more dependent on Handler on any methods after refactor. All invocations are made by handler to the FSRemover now. Also all callbacks such as `beforeExecutingCleaningRequest`, `afterExecutingCleaningRequest`, `failureExecutingCleaningRequest` are removed from handler 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  protected final MetadataCache metadataCache;
+  protected final FSRemover fsRemover;
+  protected final ExecutorService cleanerExecutor;
+
+  RequestHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
+                         boolean metricsEnabled, FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    this.metadataCache = metadataCache;
+    this.metricsEnabled = metricsEnabled;
+    this.fsRemover = fsRemover;
+    this.cleanerExecutor = cleanerExecutor;
+  }
+
+  protected abstract List<Runnable> fetchCleanTasks() throws MetaException;

Review Comment:
   `getTasks()`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  protected final MetadataCache metadataCache;
+  protected final FSRemover fsRemover;
+  protected final ExecutorService cleanerExecutor;
+
+  RequestHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,
+                         boolean metricsEnabled, FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    this.metadataCache = metadataCache;
+    this.metricsEnabled = metricsEnabled;
+    this.fsRemover = fsRemover;
+    this.cleanerExecutor = cleanerExecutor;
+  }
+
+  protected abstract List<Runnable> fetchCleanTasks() throws MetaException;
+
+  public void process() throws Exception {

Review Comment:
   As discussed, process() is not required now since the executor service is used within the cleaner itself. Done.



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandler.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.txn.TxnStore;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class RequestHandler {

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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandlerFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A factory class to fetch handlers.
+ */
+public class RequestHandlerFactory {
+  private static final RequestHandlerFactory INSTANCE = new RequestHandlerFactory();
+
+  public static RequestHandlerFactory getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Factory class, no need to expose constructor.
+   */
+  private RequestHandlerFactory() {
+  }
+
+  public List<RequestHandler> getHandlers(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache,

Review Comment:
   This gets the handlers, so this method is required.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [4 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [1 Security Hotspot](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [36 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [34 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler {

Review Comment:
   Nice idea. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -141,45 +85,29 @@ public void run() {
                     new CleanerCycleUpdater(MetricsConstants.COMPACTION_CLEANER_CYCLE_DURATION, startedAt));
           }
 
-          long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
-
-          checkInterrupt();
-
-          List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
+          for (Handler handler : handlers) {
+            List<CleaningRequest> readyToClean = handler.findReadyToClean();

Review Comment:
   Introduced try-catch block for every handler iteration. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private final String runAs;
+  private final String cleanerMetric;
+  private final String dbName;
+  private final String tableName;
+  private final String partitionName;
+  private final boolean dropPartition;
+  private final String fullPartitionName;
+
+  public CleaningRequest(CleaningRequestBuilder<? extends CleaningRequestBuilder<?>> builder) {
+    this.type = builder.type;
+    this.location = builder.location;
+    this.obsoleteDirs = builder.obsoleteDirs;
+    this.purge = builder.purge;
+    this.fs = builder.fs;
+    this.runAs = builder.runAs;
+    this.cleanerMetric = builder.cleanerMetric;
+    this.dbName = builder.dbName;

Review Comment:
   I don't see CompactionCleaningRequest anymore. but i think your comment is still valid



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {
+    if (tableCacheOn) {
+      metaCache = Optional.of(CacheBuilder.newBuilder().softValues().build());
+    }
+    return metaCache;
+  }
+
+  public void invalidateMetaCache() {

Review Comment:
   simply `invalidate` since we already now it's a cache object



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +46,28 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<RequestHandler> requestHandlers;
 
   @Override
   public void init(AtomicBoolean stop) throws Exception {
     super.init(stop);
-    replChangeManager = ReplChangeManager.getInstance(conf);
     checkInterval = conf.getTimeVar(
             HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, TimeUnit.MILLISECONDS);
-    cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
-            conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_THREADS_NUM),
-            COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
     metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
         MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
-    boolean tableCacheOn = MetastoreConf.getBoolVar(conf,
-            MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
-    initializeCache(tableCacheOn);
+    if (CollectionUtils.isEmpty(requestHandlers)) {

Review Comment:
   Yes this is required for tests



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final HiveConf conf;
+  private final ReplChangeManager replChangeManager;
+
+  public FSRemover(HiveConf conf, ReplChangeManager replChangeManager) {
+    this.conf = conf;
+    this.replChangeManager = replChangeManager;
+  }
+
+  public List<Path> clean(CleaningRequest cr) throws MetaException {
+    Ref<List<Path>> removedFiles = Ref.from(new ArrayList<>());
+    try {
+      Callable<List<Path>> cleanUpTask;
+      cleanUpTask = () -> removeFiles(cr);
+
+      if (CompactorUtil.runJobAsSelf(cr.runAs())) {
+        removedFiles.value = cleanUpTask.call();
+      } else {
+        LOG.info("Cleaning as user {} for {}", cr.runAs(), cr.getFullPartitionName());
+        UserGroupInformation ugi = UserGroupInformation.createProxyUser(cr.runAs(),
+                UserGroupInformation.getLoginUser());
+        try {
+          ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
+            removedFiles.value = cleanUpTask.call();
+            return null;
+          });
+        } finally {
+          try {
+            FileSystem.closeAllForUGI(ugi);
+          } catch (IOException exception) {
+            LOG.error("Could not clean up file-system handles for UGI: {} for {}",
+                    ugi, cr.getFullPartitionName(), exception);
+          }
+        }
+      }
+    } catch (Exception ex) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", cr,
+              StringUtils.stringifyException(ex));
+    }
+    return removedFiles.value;
+  }
+
+  /**
+   * @param cr Cleaning request
+   * @return List of deleted files if any files were removed
+   */
+  private List<Path> removeFiles(CleaningRequest cr)
+          throws MetaException, IOException {
+    List<Path> deleted = new ArrayList<>();
+    if (cr.getObsoleteDirs().isEmpty()) {
+      return deleted;
+    }
+    LOG.info("About to remove {} obsolete directories from {}. {}", cr.getObsoleteDirs().size(),
+            cr.getLocation(), CompactorUtil.getDebugInfo(cr.getObsoleteDirs()));
+    boolean needCmRecycle;
+    try {
+      Database db = getMSForConf(conf).getDatabase(getDefaultCatalog(conf), cr.getDbName());

Review Comment:
   Added code to fetch database objects from cache if available. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

Review Comment:
   Cache is initialized based on a config - 
   In the current master this is code - 
   https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java#L114-L116
   
   I think accepting a boolean in constructor and initializing might also make sense. Do you think we should do 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] deniskuzZ commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -212,323 +138,9 @@ public void run() {
     }
   }
 
-  private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
-    LOG.info("Starting cleaning for " + ci);
-    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
-    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
-        (ci.type != null ? ci.type.toString().toLowerCase() : null);
-    try {
-      if (metricsEnabled) {
-        perfLogger.perfLogBegin(CLASS_NAME, cleanerMetric);
-      }
-      final String location = ci.getProperty("location");
-
-      Callable<Boolean> cleanUpTask;
-      Table t = null;
-      Partition p = null;
-
-      if (location == null) {
-        t = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-        if (t == null) {
-          // The table was dropped before we got around to cleaning it.
-          LOG.info("Unable to find table " + ci.getFullTableName() + ", assuming it was dropped." +
-            idWatermark(ci));
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) {
-          // The table was marked no clean up true.
-          LOG.info("Skipping table " + ci.getFullTableName() + " clean up, as NO_CLEANUP set to true");
-          txnHandler.markCleaned(ci);
-          return;
-        }
-        if (ci.partName != null) {
-          p = resolvePartition(ci);
-          if (p == null) {
-            // The partition was dropped before we got around to cleaning it.
-            LOG.info("Unable to find partition " + ci.getFullPartitionName() +
-              ", assuming it was dropped." + idWatermark(ci));
-            txnHandler.markCleaned(ci);
-            return;
-          }
-          if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) {
-            // The partition was marked no clean up true.
-            LOG.info("Skipping partition " + ci.getFullPartitionName() + " clean up, as NO_CLEANUP set to true");
-            txnHandler.markCleaned(ci);
-            return;
-          }
-        }
-      }
-      txnHandler.markCleanerStart(ci);
-
-      if (t != null || ci.partName != null) {
-        String path = location == null
-            ? resolveStorageDescriptor(t, p).getLocation()
-            : location;
-        boolean dropPartition = ci.partName != null && p == null;
-        cleanUpTask = () -> removeFiles(path, minOpenTxnGLB, ci, dropPartition);
-      } else {
-        cleanUpTask = () -> removeFiles(location, ci);
-      }
-
-      Ref<Boolean> removedFiles = Ref.from(false);
-      if (runJobAsSelf(ci.runAs)) {
-        removedFiles.value = cleanUpTask.call();
-      } else {
-        LOG.info("Cleaning as user " + ci.runAs + " for " + ci.getFullPartitionName());
-        UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs,
-            UserGroupInformation.getLoginUser());
-        try {
-          ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-            removedFiles.value = cleanUpTask.call();
-            return null;
-          });
-        } finally {
-          try {
-            FileSystem.closeAllForUGI(ugi);
-          } catch (IOException exception) {
-            LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " +
-                ci.getFullPartitionName() + idWatermark(ci), exception);
-          }
-        }
-      }
-      if (removedFiles.value || isDynPartAbort(t, ci)) {
-        txnHandler.markCleaned(ci);
-      } else {
-        txnHandler.clearCleanerStart(ci);
-        LOG.warn("No files were removed. Leaving queue entry " + ci + " in ready for cleaning state.");
-      }
-    } catch (Exception e) {
-      LOG.error("Caught exception when cleaning, unable to complete cleaning of " + ci + " " +
-          StringUtils.stringifyException(e));
-      ci.errorMessage = e.getMessage();
-      if (metricsEnabled) {
-        Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
-      }
-      handleCleanerAttemptFailure(ci);
-    }  finally {
-      if (metricsEnabled) {
-        perfLogger.perfLogEnd(CLASS_NAME, cleanerMetric);
-      }
-    }
-  }
-
-  private void handleCleanerAttemptFailure(CompactionInfo ci) throws MetaException {
-    long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS);
-    int cleanAttempts = 0;
-    if (ci.retryRetention > 0) {
-      cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1;
-    }
-    if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) {
-      //Mark it as failed if the max attempt threshold is reached.
-      txnHandler.markFailed(ci);
-    } else {
-      //Calculate retry retention time and update record.
-      ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention;
-      txnHandler.setCleanerRetryRetentionTimeOnError(ci);
-    }
-  }
-
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)
-      throws NoSuchTxnException, MetaException {
-    List<String> tblNames = Collections.singletonList(AcidUtils.getFullTableName(ci.dbname, ci.tableName));
-    GetValidWriteIdsRequest request = new GetValidWriteIdsRequest(tblNames);
-    request.setValidTxnList(validTxnList.writeToString());
-    GetValidWriteIdsResponse rsp = txnHandler.getValidWriteIds(request);
-    // we could have no write IDs for a table if it was never written to but
-    // since we are in the Cleaner phase of compactions, there must have
-    // been some delta/base dirs
-    assert rsp != null && rsp.getTblValidWriteIdsSize() == 1;
-    ValidReaderWriteIdList validWriteIdList =
-        TxnCommonUtils.createValidReaderWriteIdList(rsp.getTblValidWriteIds().get(0));
-    /*
-     * We need to filter the obsoletes dir list, to only remove directories that were made obsolete by this compaction
-     * If we have a higher retentionTime it is possible for a second compaction to run on the same partition. Cleaning up the first compaction
-     * should not touch the newer obsolete directories to not to violate the retentionTime for those.
-     */
-    if (ci.highestWriteId < validWriteIdList.getHighWatermark()) {
-      validWriteIdList = validWriteIdList.updateHighWatermark(ci.highestWriteId);
-    }
-    return validWriteIdList;
-  }
-
-  private static boolean isDynPartAbort(Table t, CompactionInfo ci) {
-    return Optional.ofNullable(t).map(Table::getPartitionKeys).filter(pk -> pk.size() > 0).isPresent()
-        && ci.partName == null;
-  }
-
-  private static String idWatermark(CompactionInfo ci) {
-    return " id=" + ci.id;
-  }
-
-  private boolean removeFiles(String location, long minOpenTxnGLB, CompactionInfo ci, boolean dropPartition)
-      throws Exception {
-
-    if (dropPartition) {
-      LockRequest lockRequest = createLockRequest(ci, 0, LockType.EXCL_WRITE, DataOperationType.DELETE);
-      LockResponse res = null;
-
-      try {
-        res = txnHandler.lock(lockRequest);
-        if (res.getState() == LockState.ACQUIRED) {
-          //check if partition wasn't re-created
-          if (resolvePartition(ci) == null) {
-            return removeFiles(location, ci);
-          }
-        }
-      } catch (NoSuchTxnException | TxnAbortedException e) {
-        LOG.error(e.getMessage());
-      } finally {
-        if (res != null) {
-          try {
-            txnHandler.unlock(new UnlockRequest(res.getLockid()));
-          } catch (NoSuchLockException | TxnOpenException e) {
-            LOG.error(e.getMessage());
-          }
-        }
-      }
-    }
-
-    ValidTxnList validTxnList =
-      TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB);
-    //save it so that getAcidState() sees it
-    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
-    /**
-     * {@code validTxnList} is capped by minOpenTxnGLB so if
-     * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
-     * produced by a compactor, that means every reader that could be active right now see it
-     * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
-     * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
-     *
-     *
-     * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
-     * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
-     * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
-     * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
-     * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
-     * writeId:17.  Compactor will produce base_17_c160.
-     * Suppose txnid:149 writes delta_18_18
-     * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
-     * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
-     * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
-     * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
-     * metadata that says that 18 is aborted.
-     * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
-     * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
-     * it (since it has nothing but aborted data).  But we can't tell which actually happened
-     * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
-     *
-     * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
-     * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
-     * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
-     * useful if there is all of a sudden a flood of aborted txns.  (For another day).
-     */
-
-    // 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);
-
-    return removeFiles(location, validWriteIdList, ci);
-  }
-  /**
-   * @return true if any files were removed
-   */
-  private boolean removeFiles(String location, ValidWriteIdList writeIdList, CompactionInfo ci)
-      throws Exception {
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-    
-    // Collect all of the files/dirs
-    Map<Path, AcidUtils.HdfsDirSnapshot> dirSnapshots = AcidUtils.getHdfsDirSnapshotsForCleaner(fs, path);
-    AcidDirectory dir = AcidUtils.getAcidState(fs, path, conf, writeIdList, Ref.from(false), false, 
-        dirSnapshots);
-    Table table = computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci));
-    boolean isDynPartAbort = isDynPartAbort(table, ci);
-    
-    List<Path> obsoleteDirs = getObsoleteDirs(dir, isDynPartAbort);
-    if (isDynPartAbort || dir.hasUncompactedAborts()) {
-      ci.setWriteIds(dir.hasUncompactedAborts(), dir.getAbortedWriteIds());
-    }
-    List<Path> deleted = remove(location, ci, obsoleteDirs, true, fs);
-    if (dir.getObsolete().size() > 0) {
-      AcidMetricService.updateMetricsFromCleaner(ci.dbname, ci.tableName, ci.partName, dir.getObsolete(), conf,
-          txnHandler);
-    }
-    // Make sure there are no leftovers below the compacted watermark
-    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);
-    
-    List<Path> remained = subtract(getObsoleteDirs(dir, isDynPartAbort), deleted);
-    if (!remained.isEmpty()) {
-      LOG.warn(idWatermark(ci) + " Remained " + remained.size() +
-        " obsolete directories from " + location + ". " + getDebugInfo(remained));
-      return false;
-    }
-    LOG.debug(idWatermark(ci) + " All cleared below the watermark: " + ci.highestWriteId + " from " + location);
-    return true;
-  }
-  
-  private List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
-    List<Path> obsoleteDirs = dir.getObsolete();
-    /**
-     * add anything in 'dir'  that only has data from aborted transactions - no one should be
-     * trying to read anything in that dir (except getAcidState() that only reads the name of
-     * this dir itself)
-     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
-     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
-     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
-     * See {@link TxnStore#markCleaned(CompactionInfo)}
-     */
-    obsoleteDirs.addAll(dir.getAbortedDirectories());
-    if (isDynPartAbort) {
-      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
-      // Including obsolete directories for partitioned tables can result in data loss.
-      obsoleteDirs = dir.getAbortedDirectories();
-    }
-    return obsoleteDirs;
-  }
-
-  private boolean removeFiles(String location, CompactionInfo ci) throws IOException, MetaException {
-    String strIfPurge = ci.getProperty("ifPurge");
-    boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
-    
-    Path path = new Path(location);
-    return !remove(location, ci, Collections.singletonList(path), ifPurge,
-      path.getFileSystem(conf)).isEmpty();
-  }
-
-  private List<Path> remove(String location, CompactionInfo ci, List<Path> paths, boolean ifPurge, FileSystem fs)
-      throws MetaException, IOException {
-    List<Path> deleted = new ArrayList<>();
-    if (paths.size() < 1) {
-      return deleted;
-    }
-    LOG.info(idWatermark(ci) + " About to remove " + paths.size() +
-      " obsolete directories from " + location + ". " + getDebugInfo(paths));
-    boolean needCmRecycle;
-    try {
-      Database db = getMSForConf(conf).getDatabase(getDefaultCatalog(conf), ci.dbname);
-      needCmRecycle = ReplChangeManager.isSourceOfReplication(db);
-    } catch (NoSuchObjectException ex) {
-      // can not drop a database which is a source of replication
-      needCmRecycle = false;
-    }
-    for (Path dead : paths) {
-      LOG.debug("Going to delete path " + dead.toString());
-      if (needCmRecycle) {
-        replChangeManager.recycle(dead, ReplChangeManager.RecycleType.MOVE, ifPurge);
-      }
-      if (FileUtils.moveToTrash(fs, dead, conf, ifPurge)) {
-        deleted.add(dead);
-      }
-    }
-    return deleted;
-  }
-  
-  private String getDebugInfo(List<Path> paths) {
-    return "[" + paths.stream().map(Path::getName).collect(Collectors.joining(",")) + ']';
+  @VisibleForTesting
+  public void setCleaningRequestHandlers(List<RequestHandler> requestHandlers) {

Review Comment:
   simply `setRequestHandlers`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects of this class are created by request handlers.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {

Review Comment:
   maybe `CleanupRequest`? up to you



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


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


[GitHub] [hive] deniskuzZ commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

Review Comment:
   what is the usage pattern for this, do we use the return type anywhere? if not, can we initialize metaCache in the constructor?



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleanHandler.java:
##########
@@ -0,0 +1,391 @@
+/*
+ * 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.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+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.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CleanupRequest;
+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.CompactorUtil.ThrowingRunnable;
+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.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+import static java.util.Objects.isNull;
+
+/**
+ * A compaction based implementation of RequestHandler.
+ * Provides implementation of creation of compaction clean tasks.
+ */
+class CompactionCleanHandler extends RequestHandler {

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] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.Handler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in handler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover implements ThrowingRunnable<MetaException> {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Handler handler;
+  private final CleaningRequest cr;
+  private final ReplChangeManager replChangeManager;
+
+  public FSRemover(Handler handler, CleaningRequest cr) throws MetaException {
+    this.handler = handler;
+    this.cr = cr;
+    this.replChangeManager = ReplChangeManager.getInstance(this.handler.getConf());

Review Comment:
   Passed it as a constructor argument. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Handler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  private Optional<Cache<String, TBase>> metaCache;
+
+  Handler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    boolean tableCacheOn = MetastoreConf.getBoolVar(this.conf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
+    this.metaCache = initializeCache(tableCacheOn);
+    this.metricsEnabled = metricsEnabled;
+  }
+
+  public HiveConf getConf() {
+    return conf;
+  }
+
+  public TxnStore getTxnHandler() {
+    return txnHandler;
+  }
+
+  public boolean isMetricsEnabled() {
+    return metricsEnabled;
+  }
+
+  /**
+   * Find the list of objects which are ready for cleaning.
+   * @return Cleaning requests
+   */
+  public abstract List<CleaningRequest> findReadyToClean() throws MetaException;
+
+  /**
+   * Execute just before cleanup
+   * @param cleaningRequest - Cleaning request
+   */
+  public abstract void beforeExecutingCleaningRequest(CleaningRequest cleaningRequest) throws MetaException;
+
+  /**
+   * Execute just after cleanup
+   * @param cleaningRequest Cleaning request
+   * @param deletedFiles List of deleted files
+   * @throws MetaException
+   */
+  public abstract void afterExecutingCleaningRequest(CleaningRequest cleaningRequest, List<Path> deletedFiles) throws MetaException;
+
+  /**
+   * Execute in the event of failure
+   * @param cleaningRequest Cleaning request
+   * @param ex Failure exception
+   * @throws MetaException
+   */
+  public abstract void failureExecutingCleaningRequest(CleaningRequest cleaningRequest, Exception ex) throws MetaException;

Review Comment:
   Discussed it with @veghlaci05 and concluded that this might not be required right now, since handler must be aware of what needs to be done in case of failure and this doesnt require notifying all observers in case of exception.



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


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


[GitHub] [hive] SourabhBadhya commented on a diff in pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/RequestHandlerFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A factory class to fetch handlers.
+ */
+public class RequestHandlerFactory {

Review Comment:
   This gets the TaskHandlers so, I have renamed it as `TaskHandlerFactory`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private final String runAs;
+  private final String cleanerMetric;

Review Comment:
   This is dependent on compaction type - 
   https://github.com/apache/hive/pull/4032/files#diff-29a4f98737540a5a074932f2fe51df18c9e145eb5cfb6cd831834acf641bc02cR81-R82



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionHandler.java:
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+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.TxnCommonUtils;
+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.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.CacheContainer;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest.CleaningRequestBuilder;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.commons.collections4.ListUtils.subtract;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar;
+import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar;
+import static java.util.Objects.isNull;
+
+/**
+ * A compaction based implementation of RequestHandler.
+ * Provides implementation of finding ready to clean items, preprocessing of cleaning request,
+ * postprocessing of cleaning request and failure handling of cleaning request.
+ */
+class CompactionHandler extends RequestHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionHandler.class.getName());
+
+  public CompactionHandler(HiveConf conf, TxnStore txnHandler,
+                                          CacheContainer cacheContainer, boolean metricsEnabled,
+                                          FSRemover fsRemover, ExecutorService cleanerExecutor) {
+    super(conf, txnHandler, cacheContainer, metricsEnabled, fsRemover, cleanerExecutor);
+  }
+
+  @Override
+  protected void processInternal() throws Exception {
+    long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
+    long retentionTime = HiveConf.getBoolVar(conf, HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED)
+            ? HiveConf.getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETENTION_TIME, TimeUnit.MILLISECONDS)
+            : 0;
+    List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
+    if (!readyToClean.isEmpty()) {
+      long minTxnIdSeenOpen = txnHandler.findMinTxnIdSeenOpen();
+      final long cleanerWaterMark =
+              minTxnIdSeenOpen < 0 ? minOpenTxnId : Math.min(minOpenTxnId, minTxnIdSeenOpen);
+
+      LOG.info("Cleaning based on min open txn id: {}", cleanerWaterMark);
+      // For checking which compaction can be cleaned we can use the minOpenTxnId
+      // However findReadyToClean will return all records that were compacted with old version of HMS
+      // where the CQ_NEXT_TXN_ID is not set. For these compactions we need to provide minTxnIdSeenOpen
+      // to the clean method, to avoid cleaning up deltas needed for running queries
+      // when min_history_level is finally dropped, than every HMS will commit compaction the new way
+      // and minTxnIdSeenOpen can be removed and minOpenTxnId can be used instead.
+      List<CompletableFuture<Void>> asyncTasks = new ArrayList<>();
+      for (CompactionInfo ci : readyToClean) {
+        CompletableFuture<Void> asyncTask = CompletableFuture.runAsync(
+                CompactorUtil.ThrowingRunnable.unchecked(() -> clean(ci, cleanerWaterMark, metricsEnabled)), cleanerExecutor)
+                .exceptionally(t -> {
+                  LOG.error("Error clearing {} due to :", ci.getFullPartitionName(), t);
+                  return null;
+                });
+        asyncTasks.add(asyncTask);
+      }
+      //Use get instead of join, so we can receive InterruptedException and shutdown gracefully
+      CompletableFuture.allOf(asyncTasks.toArray(new CompletableFuture[0])).get();
+    }
+  }
+
+  private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
+    LOG.info("Starting cleaning for {}", ci);
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
+            (!isNull(ci.type) ? ci.type.toString().toLowerCase() : null);
+    try {
+      if (metricsEnabled) {
+        perfLogger.perfLogBegin(CompactionHandler.class.getName(), cleanerMetric);
+      }
+      final String location = ci.getProperty("location");
+
+      Table t = null;
+      Partition p = null;
+
+      if (isNull(location)) {
+        t = cacheContainer.computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci.dbname, ci.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. {}", ci.getFullTableName(),
+                  idWatermark(ci));
+          txnHandler.markCleaned(ci);
+          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", ci.getFullTableName());
+          txnHandler.markCleaned(ci);
+          return;
+        }
+        if (!isNull(ci.partName)) {
+          p = resolvePartition(ci.dbname, ci.tableName, ci.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. {}",
+                    ci.getFullPartitionName(), idWatermark(ci));
+            txnHandler.markCleaned(ci);
+            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", ci.getFullPartitionName());
+            txnHandler.markCleaned(ci);
+            return;
+          }
+        }
+      }
+      txnHandler.markCleanerStart(ci);
+
+      if (!isNull(t) || !isNull(ci.partName)) {
+        String path = isNull(location)
+                ? CompactorUtil.resolveStorageDescriptor(t, p).getLocation()
+                : location;
+        boolean dropPartition = !isNull(ci.partName) && isNull(p);
+
+        //check if partition wasn't re-created
+        if (dropPartition && isNull(resolvePartition(ci.dbname, ci.tableName, ci.partName))) {
+          cleanUsingLocation(ci, path, true);
+        } else {
+          cleanUsingAcidDir(ci, path, minOpenTxnGLB);
+        }
+      } else {
+        cleanUsingLocation(ci, location, false);
+      }
+    } catch (Exception e) {
+      LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", ci,
+              e.getMessage());
+      ci.errorMessage = e.getMessage();
+      if (metricsEnabled) {
+        Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc();
+      }
+      handleCleanerAttemptFailure(ci);
+    }  finally {
+      if (metricsEnabled) {
+        perfLogger.perfLogEnd(CompactionHandler.class.getName(), cleanerMetric);
+      }
+    }
+  }
+
+  private void cleanUsingLocation(CompactionInfo ci, String path, boolean requiresLock) throws MetaException {
+    List<Path> deleted;
+    if (requiresLock) {
+      LockRequest lockRequest = createLockRequest(ci, 0, LockType.EXCL_WRITE, DataOperationType.DELETE);
+      LockResponse res = null;
+      try {
+        res = txnHandler.lock(lockRequest);
+        deleted = fsRemover.clean(getCleaningRequestBasedOnLocation(ci, path));
+      } catch (NoSuchTxnException | TxnAbortedException e) {
+        LOG.error("Error while trying to acquire exclusive write lock: {}", e.getMessage());
+        throw new MetaException(e.getMessage());
+      } finally {
+        if (res != null) {
+          try {
+            txnHandler.unlock(new UnlockRequest(res.getLockid()));
+          } catch (NoSuchLockException | TxnOpenException e) {
+            LOG.error("Error while trying to release exclusive write lock: {}", e.getMessage());
+          }
+        }
+      }
+    } else {
+      deleted = fsRemover.clean(getCleaningRequestBasedOnLocation(ci, path));
+    }
+    if (!deleted.isEmpty()) {
+      txnHandler.markCleaned(ci);
+    } else {
+      txnHandler.clearCleanerStart(ci);
+    }
+  }
+
+  private void cleanUsingAcidDir(CompactionInfo ci, String location, long minOpenTxnGLB) throws Exception {
+    ValidTxnList validTxnList =
+            TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB);
+    //save it so that getAcidState() sees it
+    conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
+    /*
+     * {@code validTxnList} is capped by minOpenTxnGLB so if
+     * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
+     * produced by a compactor, that means every reader that could be active right now see it
+     * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
+     * used in favor of any files that it shadows.  Thus the shadowed files are safe to delete.
+     *
+     *
+     * The metadata about aborted writeIds (and consequently aborted txn IDs) cannot be deleted
+     * above COMPACTION_QUEUE.CQ_HIGHEST_WRITE_ID.
+     * See {@link TxnStore#markCleaned(CompactionInfo)} for details.
+     * For example given partition P1, txnid:150 starts and sees txnid:149 as open.
+     * Say compactor runs in txnid:160, but 149 is still open and P1 has the largest resolved
+     * writeId:17.  Compactor will produce base_17_c160.
+     * Suppose txnid:149 writes delta_18_18
+     * to P1 and aborts.  Compactor can only remove TXN_COMPONENTS entries
+     * up to (inclusive) writeId:17 since delta_18_18 may be on disk (and perhaps corrupted) but
+     * not visible based on 'validTxnList' capped at minOpenTxn so it will not not be cleaned by
+     * {@link #removeFiles(String, ValidWriteIdList, CompactionInfo)} and so we must keep the
+     * metadata that says that 18 is aborted.
+     * In a slightly different case, whatever txn created delta_18 (and all other txn) may have
+     * committed by the time cleaner runs and so cleaner will indeed see delta_18_18 and remove
+     * it (since it has nothing but aborted data).  But we can't tell which actually happened
+     * in markCleaned() so make sure it doesn't delete meta above CG_CQ_HIGHEST_WRITE_ID.
+     *
+     * We could perhaps make cleaning of aborted and obsolete and remove all aborted files up
+     * to the current Min Open Write Id, this way aborted TXN_COMPONENTS meta can be removed
+     * as well up to that point which may be higher than CQ_HIGHEST_WRITE_ID.  This could be
+     * useful if there is all of a sudden a flood of aborted txns.  (For another day).
+     */
+
+    // 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);
+    Table table = cacheContainer.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 CleaningRequestBuilder().setLocation(location).setRunAs(ci.runAs)
+            .setObsoleteDirs(obsoleteDirs).setPurge(true).setFullPartitionName(ci.getFullPartitionName())
+            .build());
+
+    if (deleted.size() > 0) {
+      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);
+
+    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(idWatermark(ci) + " All cleared below the watermark: " + ci.highestWriteId + " from " + location);
+      success = true;
+    }
+    if (success || CompactorUtil.isDynPartAbort(table, ci.partName)) {
+      txnHandler.markCleaned(ci);
+    } else {
+      txnHandler.clearCleanerStart(ci);
+      LOG.warn("No files were removed. Leaving queue entry {} in ready for cleaning state.", ci);
+    }
+  }
+
+  protected LockRequest createLockRequest(CompactionInfo ci, long txnId, LockType lockType, DataOperationType opType) {

Review Comment:
   This method has a single usage, all parameters except ci can be removed. 



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

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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


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


[GitHub] [hive] sonarcloud[bot] commented on pull request #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [36 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  protected String runAs;

Review Comment:
   Made the field as private. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler<T extends CleaningRequest> {

Review Comment:
   Class name could be sth more meaningful like CleaningRequestHandler



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private String runAs;
+  protected String cleanerMetric;
+  protected String dbName;
+  protected String tableName;
+  protected String partitionName;
+  protected boolean dropPartition;
+  protected String fullPartitionName;

Review Comment:
   The only place where these are written is in CompactionCleaningRequest. I don't understand why these fields are protected and set directly in CompactionCleaningRequest, while other fields are private final and set through the constructor?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/Handler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.fs.Path;
+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.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
+import org.apache.thrift.TBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * An abstract class which defines the list of utility methods for performing cleanup activities.
+ */
+public abstract class Handler<T extends CleaningRequest> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Handler.class.getName());
+  protected final TxnStore txnHandler;
+  protected final HiveConf conf;
+  protected final boolean metricsEnabled;
+  private Optional<Cache<String, TBase>> metaCache;
+
+  Handler(HiveConf conf, TxnStore txnHandler, boolean metricsEnabled) {
+    this.conf = conf;
+    this.txnHandler = txnHandler;
+    boolean tableCacheOn = MetastoreConf.getBoolVar(this.conf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_TABLECACHE_ON);
+    this.metaCache = initializeCache(tableCacheOn);
+    this.metricsEnabled = metricsEnabled;
+  }
+
+  public HiveConf getConf() {
+    return conf;
+  }
+
+  public TxnStore getTxnHandler() {
+    return txnHandler;
+  }
+
+  public boolean isMetricsEnabled() {
+    return metricsEnabled;
+  }
+
+  /**
+   * Find the list of objects which are ready for cleaning.
+   * @return Cleaning requests
+   */
+  public abstract List<T> findReadyToClean() throws MetaException;
+
+  /**
+   * Execute just before cleanup
+   * @param cleaningRequest - Cleaning request
+   */
+  public abstract void beforeExecutingCleaningRequest(T cleaningRequest) throws MetaException;
+
+  /**
+   * Execute just after cleanup
+   * @param cleaningRequest Cleaning request
+   * @param deletedFiles List of deleted files
+   * @return True if cleanup was successful, false otherwise
+   * @throws MetaException
+   */
+  public abstract boolean afterExecutingCleaningRequest(T cleaningRequest, List<Path> deletedFiles) throws MetaException;
+
+  /**
+   * Execute in the event of failure
+   * @param cleaningRequest Cleaning request
+   * @param ex Failure exception
+   * @throws MetaException
+   */
+  public abstract void failureExecutingCleaningRequest(T cleaningRequest, Exception ex) throws MetaException;
+
+  public Table resolveTable(String dbName, String tableName) throws MetaException {
+    try {
+      return getMSForConf(conf).getTable(getDefaultCatalog(conf), dbName, tableName);
+    } catch (MetaException e) {
+      LOG.error("Unable to find table {}.{}, {}", dbName, tableName, e.getMessage());
+      throw e;
+    }
+  }
+
+  protected Partition resolvePartition(String dbName, String tableName, String partName) throws MetaException {
+    if (partName != null) {
+      List<Partition> parts;
+      try {
+        parts = CompactorUtil.getPartitionsByNames(conf, dbName, tableName, partName);
+        if (parts == null || parts.isEmpty()) {
+          // The partition got dropped before we went looking for it.
+          return null;
+        }
+      } catch (Exception e) {
+        LOG.error("Unable to find partition: {}.{}.{}", dbName, tableName, partName, e);
+        throw e;
+      }
+      if (parts.size() != 1) {
+        LOG.error("{}.{}.{} does not refer to a single partition. {}", dbName, tableName, partName,
+                Arrays.toString(parts.toArray()));
+        throw new MetaException(String.join("Too many partitions for : ", dbName, tableName, partName));
+      }
+      return parts.get(0);
+    } else {
+      return null;
+    }
+  }
+
+  public <B extends TBase<B,?>> B computeIfAbsent(String key, Callable<B> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (B) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {
+    if (tableCacheOn) {
+      metaCache = Optional.of(CacheBuilder.newBuilder().softValues().build());
+    }
+    return metaCache;
+  }
+
+  public void invalidateMetaCache() {
+    metaCache.ifPresent(Cache::invalidateAll);
+  }
+
+  protected List<Path> getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbort) {
+    List<Path> obsoleteDirs = dir.getObsolete();
+    /*
+     * add anything in 'dir'  that only has data from aborted transactions - no one should be
+     * trying to read anything in that dir (except getAcidState() that only reads the name of
+     * this dir itself)
+     * So this may run ahead of {@link CompactionInfo#highestWriteId} but it's ok (suppose there
+     * are no active txns when cleaner runs).  The key is to not delete metadata about aborted
+     * txns with write IDs > {@link CompactionInfo#highestWriteId}.
+     * See {@link TxnStore#markCleaned(CompactionInfo)}
+     */
+    obsoleteDirs.addAll(dir.getAbortedDirectories());
+    if (isDynPartAbort) {
+      // In the event of an aborted DP operation, we should only consider the aborted directories for cleanup.
+      // Including obsolete directories for partitioned tables can result in data loss.
+      obsoleteDirs = dir.getAbortedDirectories();
+    }
+    return obsoleteDirs;
+  }

Review Comment:
   Im not sure if these methods should be common. Will the AbortedTxnHandler use all of them?



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -141,49 +93,37 @@ public void run() {
                     new CleanerCycleUpdater(MetricsConstants.COMPACTION_CLEANER_CYCLE_DURATION, startedAt));
           }
 
-          long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
-
-          checkInterrupt();
-
-          List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
-
-          checkInterrupt();
-
-          if (!readyToClean.isEmpty()) {
-            long minTxnIdSeenOpen = txnHandler.findMinTxnIdSeenOpen();
-            final long cleanerWaterMark =
-                minTxnIdSeenOpen < 0 ? minOpenTxnId : Math.min(minOpenTxnId, minTxnIdSeenOpen);
-
-            LOG.info("Cleaning based on min open txn id: " + cleanerWaterMark);
-            List<CompletableFuture<Void>> cleanerList = new ArrayList<>();
-            // For checking which compaction can be cleaned we can use the minOpenTxnId
-            // However findReadyToClean will return all records that were compacted with old version of HMS
-            // where the CQ_NEXT_TXN_ID is not set. For these compactions we need to provide minTxnIdSeenOpen
-            // to the clean method, to avoid cleaning up deltas needed for running queries
-            // when min_history_level is finally dropped, than every HMS will commit compaction the new way
-            // and minTxnIdSeenOpen can be removed and minOpenTxnId can be used instead.
-            for (CompactionInfo compactionInfo : readyToClean) {
-
-              //Check for interruption before scheduling each compactionInfo and return if necessary
+          for (Handler handler : handlers) {
+            try {
+              List<CleaningRequest> readyToClean = handler.findReadyToClean();
               checkInterrupt();
 
-              CompletableFuture<Void> asyncJob =
-                  CompletableFuture.runAsync(
-                          ThrowingRunnable.unchecked(() -> clean(compactionInfo, cleanerWaterMark, metricsEnabled)),
-                          cleanerExecutor)
-                      .exceptionally(t -> {
-                        LOG.error("Error clearing {}", compactionInfo.getFullPartitionName(), t);
-                        return null;
-                      });
-              cleanerList.add(asyncJob);
+              if (!readyToClean.isEmpty()) {
+                List<CompletableFuture<Void>> cleanerList = new ArrayList<>();
+                for (CleaningRequest cr : readyToClean) {
+
+                  //Check for interruption before scheduling each cleaning request and return if necessary
+                  checkInterrupt();
+
+                  CompletableFuture<Void> asyncJob = CompletableFuture.runAsync(
+                                  ThrowingRunnable.unchecked(new FSRemover(handler, cr)), cleanerExecutor)
+                          .exceptionally(t -> {
+                            LOG.error("Error clearing: {}", cr.getFullPartitionName(), t);
+                            return null;
+                          });

Review Comment:
   This construct means that each request will have it's own FSRemover, but the handler will be common. It would be better if the "scope" of the two services would be the same, in other words: There should be a single FSRemover instance in Cleaner, it's run method could be renamed to clean and accept a CleaningRequest. The wrapping for the exception handling could be done outside:
   `ThrowingRunnable.unchecked(() -> remover.clean(cr)), cleanerExecutor)`



##########
ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestHandler.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.Cleaner;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCleaner;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+
+public class TestHandler extends TestCleaner {
+
+  @Test
+  public void testCompactionHandlerForSuccessfulCompaction() throws Exception {
+    Table t = newTable("default", "handler_success_table", true);
+    Partition p = newPartition(t, "today");
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
+
+    burnThroughTransactions(t.getDbName(), t.getTableName(), 25);
+
+    CompactionRequest rqst = new CompactionRequest(t.getDbName(), t.getTableName(), CompactionType.MAJOR);
+    rqst.setPartitionname("ds=today");
+    compactInTxn(rqst);
+
+    Handler handler = new CompactionHandler(conf, txnHandler, false);
+
+    // Fetch the compaction request using the handler
+    List<CleaningRequest> cleaningRequests = handler.findReadyToClean();
+    Assert.assertEquals(1, cleaningRequests.size());
+    CleaningRequest cr = cleaningRequests.get(0);
+    Assert.assertEquals(t.getDbName(), cr.getDbName());
+    Assert.assertEquals(t.getTableName(), cr.getTableName());
+    Assert.assertEquals("ds=today", cr.getPartitionName());
+    Assert.assertEquals(CleaningRequest.RequestType.COMPACTION, cr.getType());
+
+    // Check whether appropriate handler utility methods are called exactly once in a successful compaction scenario.
+    Handler mockedHandler = Mockito.spy(handler);
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Cleaner cleaner = new Cleaner(Arrays.asList(mockedHandler));
+    cleaner.setConf(conf);
+    cleaner.init(stop);
+    cleaner.run();
+
+    Mockito.verify(mockedHandler, Mockito.times(1)).findReadyToClean();
+    Mockito.verify(mockedHandler, Mockito.times(1)).beforeExecutingCleaningRequest(any(CleaningRequest.class));
+    Mockito.verify(mockedHandler, Mockito.times(1)).afterExecutingCleaningRequest(any(CleaningRequest.class), any(List.class));

Review Comment:
   You could also assert that failureExecutingCleaningRequest was not invoked.



##########
ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestHandler.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.ql.txn.compactor.Cleaner;
+import org.apache.hadoop.hive.ql.txn.compactor.CleaningRequest;
+import org.apache.hadoop.hive.ql.txn.compactor.TestCleaner;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+
+public class TestHandler extends TestCleaner {
+
+  @Test
+  public void testCompactionHandlerForSuccessfulCompaction() throws Exception {
+    Table t = newTable("default", "handler_success_table", true);
+    Partition p = newPartition(t, "today");
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
+
+    burnThroughTransactions(t.getDbName(), t.getTableName(), 25);
+
+    CompactionRequest rqst = new CompactionRequest(t.getDbName(), t.getTableName(), CompactionType.MAJOR);
+    rqst.setPartitionname("ds=today");
+    compactInTxn(rqst);
+
+    Handler handler = new CompactionHandler(conf, txnHandler, false);
+
+    // Fetch the compaction request using the handler
+    List<CleaningRequest> cleaningRequests = handler.findReadyToClean();
+    Assert.assertEquals(1, cleaningRequests.size());
+    CleaningRequest cr = cleaningRequests.get(0);
+    Assert.assertEquals(t.getDbName(), cr.getDbName());
+    Assert.assertEquals(t.getTableName(), cr.getTableName());
+    Assert.assertEquals("ds=today", cr.getPartitionName());
+    Assert.assertEquals(CleaningRequest.RequestType.COMPACTION, cr.getType());
+
+    // Check whether appropriate handler utility methods are called exactly once in a successful compaction scenario.
+    Handler mockedHandler = Mockito.spy(handler);
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Cleaner cleaner = new Cleaner(Arrays.asList(mockedHandler));
+    cleaner.setConf(conf);
+    cleaner.init(stop);
+    cleaner.run();
+
+    Mockito.verify(mockedHandler, Mockito.times(1)).findReadyToClean();
+    Mockito.verify(mockedHandler, Mockito.times(1)).beforeExecutingCleaningRequest(any(CleaningRequest.class));
+    Mockito.verify(mockedHandler, Mockito.times(1)).afterExecutingCleaningRequest(any(CleaningRequest.class), any(List.class));
+  }
+
+  @Test
+  public void testCompactionHandlerForFailureCompaction() throws Exception {
+    Table t = newTable("default", "handler_failure_table", true);
+    Partition p = newPartition(t, "today");
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
+
+    burnThroughTransactions(t.getDbName(), t.getTableName(), 25);
+
+    CompactionRequest rqst = new CompactionRequest(t.getDbName(), t.getTableName(), CompactionType.MAJOR);
+    rqst.setPartitionname("ds=today");
+    compactInTxn(rqst);
+
+    // Check whether appropriate handler utility methods are called exactly once in a failure compaction scenario.
+    TxnStore mockedTxnHandler = Mockito.spy(txnHandler);
+    doThrow(new RuntimeException()).when(mockedTxnHandler).markCleaned(any());
+    Handler mockedHandler = Mockito.spy(new CompactionHandler(conf, mockedTxnHandler, false));
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Cleaner cleaner = new Cleaner(Arrays.asList(mockedHandler));
+    cleaner.setConf(conf);
+    cleaner.init(stop);
+    cleaner.run();
+
+    Mockito.verify(mockedHandler, Mockito.times(1)).findReadyToClean();
+    Mockito.verify(mockedHandler, Mockito.times(1)).beforeExecutingCleaningRequest(any(CleaningRequest.class));
+    Mockito.verify(mockedHandler, Mockito.times(1)).failureExecutingCleaningRequest(any(CleaningRequest.class), any(Exception.class));

Review Comment:
   You could also assert that afterExecutingCleaningRequest was not invoked.



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CleaningRequest.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * A class which specifies the required information for cleanup.
+ * Objects from this class are passed to FSRemover for cleanup.
+ */
+public class CleaningRequest {
+  public enum RequestType {
+    COMPACTION,
+  }
+  private final RequestType type;
+  private final String location;
+  private final List<Path> obsoleteDirs;
+  private final boolean purge;
+  private final FileSystem fs;
+  private String runAs;
+  protected String cleanerMetric;
+  protected String dbName;
+  protected String tableName;
+  protected String partitionName;
+  protected boolean dropPartition;
+  protected String fullPartitionName;

Review Comment:
   Made all fields `private final` and used builder implementation for constructing cleaning requests. 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [3 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [27 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CacheContainer.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class CacheContainer {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   consider using a Caffeine cache, it provides more control over a cache 



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/FSRemover.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.ql.txn.compactor.handler.CleaningRequestHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.hive.metastore.HMSHandler.getMSForConf;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
+/**
+ * A runnable class which takes in cleaningRequestHandler and cleaning request and deletes the files
+ * according to the cleaning request.
+ */
+public class FSRemover {
+  private static final Logger LOG = LoggerFactory.getLogger(FSRemover.class);
+  private final Map<CleaningRequest.RequestType, CleaningRequestHandler> handlerMap;

Review Comment:
   FSRemover is no more dependent on Handler on any methods after refactor. All invocations are made by handler to the FSRemover now. Also all callbacks such as beforeExecutingCleaningRequest, afterExecutingCleaningRequest, failureExecutingCleaningRequest are removed from handler 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +46,28 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<RequestHandler> requestHandlers;

Review Comment:
   maybe `cleanupHandlers`



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   i don't think we need to wrap it with Optional in a current design



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   @deniskuzZ Optional is required to check whether the cache is actually built or not.
   Related code in current master - https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java#L145-L154



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();

Review Comment:
   I don't think we need to wrap it with Optional in a current design, we could just check `!= 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {

Review Comment:
   What is the usage pattern for this, do we use the return type anywhere? 
   should it be simple `public void init(boolean enabled)`?
   maybe would be even better if we do this in constructor
   ````
   metadataCache = new MetadataCache(isCacheEnabled());
   ````



-- 
This is an automated message from the Apache Git Service.
To 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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java:
##########
@@ -97,23 +46,28 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private boolean metricsEnabled = false;
 
-  private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
+  private List<RequestHandler> requestHandlers;

Review Comment:
   Done



##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache = Optional.empty();
+
+  public <T extends TBase<T,?>> T computeIfAbsent(String key, Callable<T> callable) throws Exception {
+    if (metaCache.isPresent()) {
+      try {
+        return (T) metaCache.get().get(key, callable);
+      } catch (ExecutionException e) {
+        throw (Exception) e.getCause();
+      }
+    }
+    return callable.call();
+  }
+
+  public Optional<Cache<String, TBase>> initializeCache(boolean tableCacheOn) {
+    if (tableCacheOn) {
+      metaCache = Optional.of(CacheBuilder.newBuilder().softValues().build());
+    }
+    return metaCache;
+  }
+
+  public void invalidateMetaCache() {

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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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

   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=4032)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_hive&pullRequest=4032&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=4032&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=4032&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_hive&pullRequest=4032&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=4032&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=4032&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 #4032: HIVE-27019: Split Cleaner into separate manageable modular entities

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


##########
ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetadataCache.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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 com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.thrift.TBase;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+public class MetadataCache {
+
+  private Optional<Cache<String, TBase>> metaCache;

Review Comment:
   do we really need to wrap with Optional 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