You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2021/10/22 13:54:34 UTC

[GitHub] [hbase] BukrosSzabolcs opened a new pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

BukrosSzabolcs opened a new pull request #3786:
URL: https://github.com/apache/hbase/pull/3786


   Add new chore to delete lefotver files in case file based storefile
   handling is used
   Expose the target files of currently running compactions for easier
   validation


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736406688



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       > Or do you plan to move the nulling of write to a resetWriter method?
   
   I'm planing to remove the `writer = null ` lines from `commitWriter` methods, it was not very pretty to begin with and create a `resetWriter` in `Compactor` we could expose through `StoreEngine` and call in `HStore.doCompaction`.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-954578159


   The failing tests seem to be flaky. I was able to successfully run them locally.


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736312453



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,14 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));
+        writer = null;

Review comment:
       I agree with the WARN, what I meant is that we don't need to explicitly set it to null in line #360, when we just go and set it again on line #362.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736316274



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -1895,6 +1900,22 @@ private void initializeThreads() {
       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
           onlyMetaRefresh, this, this);
     }
+
+    int fileBasedStoreFileCleanerPeriod  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_PERIOD,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD);
+    int fileBasedStoreFileCleanerDelay  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY);
+    double fileBasedStoreFileCleanerDelayJitter = conf.getDouble(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY_JITTER,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER);
+    double jitterRate = (RandomUtils.nextDouble() - 0.5D) * fileBasedStoreFileCleanerDelayJitter;
+    long jitterValue = Math.round(fileBasedStoreFileCleanerDelay * jitterRate);

Review comment:
       Yeah, since we already pass the Configuration and these are all specific for this Cleaner itself, I think it's better to move it out of HRegionServer scope.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-963558439


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m 14s |  HBASE-26067 passed  |
   | +1 :green_heart: |  checkstyle  |   1m  5s |  HBASE-26067 passed  |
   | +1 :green_heart: |  spotbugs  |   2m  3s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   2m 13s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  9s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  9s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  4s |  hbase-server: The patch generated 23 new + 61 unchanged - 1 fixed = 84 total (was 62)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 27s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 15s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 14s |  The patch does not generate ASF License warnings.  |
   |  |   |  49m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux ef2af575e44e 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / a0ec946876 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-962466544


   #3700 has been merged and I rebased HBASE-26067 with the newest master.
   
   Please rebase the PR too. Thanks for your patient. We are very close to merge this PR.


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-949656014


   @Apache9, @wchevreuil  This is the commit only containing changes related to the CleanerChore and exposing what is absolutely necessary for it's functionality.


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-949697945


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m 10s |  HBASE-26067 passed  |
   | +1 :green_heart: |  checkstyle  |   1m  6s |  HBASE-26067 passed  |
   | +1 :green_heart: |  spotbugs  |   2m  8s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 47s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 11s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 11s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  5s |  hbase-server: The patch generated 22 new + 59 unchanged - 0 fixed = 81 total (was 59)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 21s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 17s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  49m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 6ffb598edba4 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 29fb33bf87 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 95 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-954145933


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  2s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 36s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m  4s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   9m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |  10m  0s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 18s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  7s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 218m 21s |  hbase-server in the patch passed.  |
   |  |   | 251m 51s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2d61df0298f6 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 3240b4b39c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/testReport/ |
   | Max. process+thread count | 3889 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736315277



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {

Review comment:
       Just another name suggestion: "LeftoversStoreFileCleaner"... Up to you, @BukrosSzabolcs !




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736402871



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(FileBasedStoreFileCleaner.class);
+  public static final String FILEBASED_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.filebased.storefilecleaner.enabled";
+  public static final boolean DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED = false;
+  public static final String FILEBASED_STOREFILE_CLEANER_TTL =
+      "hbase.region.filebased.storefilecleaner.ttl";
+  public static final long DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY =
+      "hbase.region.filebased.storefilecleaner.delay";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.filebased.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String FILEBASED_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.filebased.storefilecleaner.period";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public FileBasedStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("FileBasedStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(FILEBASED_STOREFILE_CLEANER_ENABLED, DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(FILEBASED_STOREFILE_CLEANER_TTL, DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only clean do cleanup in store using file based storefile tracking
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "FileBasedStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("File based storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {

Review comment:
       > And do we really need to test isCompactingFile here? The compactedFile will not be tracked but compacting files should always be tracked?
   
   The confusion might come from suboptimal naming.
   isActiveStorefile -> lists the currently active storefiles
   isCompactedFile -> are hfiles that got compacted, no longer active storefiles, but were not deleted yet (deletion is handled by a separate subsystem, so we should not touch them)
   isCompactingFile -> file(s) a currently running compaction is writing into. This will become the new storefile when the compaction is done. It is checked to make sure we do not break stuck/longrunning compactions even is they are stuck/idle for more than the configured TTL.
   I think we do need all of these checks.
   
   > I suppose we should move the below isXXXFile to HRegion or HStore, with the protection of some locks. Otherwise we may have race, or at least, we depend on a very flaky order of the testing of each condition.
   
   As Wellington mentioned, the TTL should make sure we filter out any currently handled hfile. Any file we check afterwards was not touched in 12 hours. I think the only usecase where this might be an issue is a stuck compaction where we check isCompactingFile, but that method (getCompactionTargets) is synchronised so should be safe. Do you see any other possible issue?




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r738496846



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -1895,6 +1900,22 @@ private void initializeThreads() {
       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
           onlyMetaRefresh, this, this);
     }
+
+    int fileBasedStoreFileCleanerPeriod  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_PERIOD,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD);
+    int fileBasedStoreFileCleanerDelay  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY);
+    double fileBasedStoreFileCleanerDelayJitter = conf.getDouble(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY_JITTER,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER);
+    double jitterRate = (RandomUtils.nextDouble() - 0.5D) * fileBasedStoreFileCleanerDelayJitter;
+    long jitterValue = Math.round(fileBasedStoreFileCleanerDelay * jitterRate);

Review comment:
       If you feel it's cleaner this way, I'm ok with that.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] joshelser commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r743071792



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,13 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));

Review comment:
       > the worst case scenario is that the cleaner would skip this file once because it would assume the compaction is still ongoing
   
   Cool, yeah, agree that is not the end of the world. `LOG.warn` seems appropriate :)




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-961149870


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 34s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m 39s |  HBASE-26067 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  HBASE-26067 passed  |
   | +1 :green_heart: |  spotbugs  |   2m 17s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   2m 27s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 20s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 30s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 10s |  hbase-server: The patch generated 23 new + 62 unchanged - 0 fixed = 85 total (was 62)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  22m  0s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  55m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux b3c8beb2b0f4 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 3240b4b39c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 95 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] joshelser commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r742417329



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {

Review comment:
       nit: javadoc

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {

Review comment:
       ```suggestion
   @InterfaceAudience.Private
   public class BrokenStoreFileCleaner extends ScheduledChore {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
##########
@@ -72,13 +72,20 @@ public StoreFileWriter createWriter(InternalScanner scanner,
   }
 
   @Override
+  protected void abortWriter() throws IOException {
+    abortWriter(writer);
+  }
+
   protected void abortWriter(StoreFileWriter writer) throws IOException {
     Path leftoverFile = writer.getPath();
     try {
       writer.close();
     } catch (IOException e) {
       LOG.warn("Failed to close the writer after an unfinished compaction.", e);
     }
+    finally {

Review comment:
       ```suggestion
       } finally {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {
+    return storeFileTracker.requireWritingToTmpDirFirst();
+  }
+
+  public void resetCompactionWriter(){

Review comment:
       And a javadoc comment here as this is critical to ensuring that we know which files are sane to be deleted, please. Touch on `doCompaction` calling this.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +546,24 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){
+      return Collections.emptyList();
+    }
+    synchronized (writer){
+      if (writer instanceof StoreFileWriter){
+        return Arrays.asList(((StoreFileWriter)writer).getPath());
+      }
+      return ((AbstractMultiFileWriter)writer).writers().stream().map(sfw -> sfw.getPath()).collect(
+        Collectors.toList());
+    }
+  }
+
+  /**
+   * Reset the Writer when the new storefiles were successfully added
+   */
+  public  void resetWriter(){

Review comment:
       ```suggestion
     public void resetWriter() {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
##########
@@ -110,7 +110,7 @@ public void init(StoreScanner sourceScanner, WriterFactory factory) {
     return paths;
   }
 
-  protected abstract Collection<StoreFileWriter> writers();
+  public abstract Collection<StoreFileWriter> writers();

Review comment:
       nit: should have Javadoc on a public method.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -1159,6 +1159,8 @@ public void deleteChangedReaderObserver(ChangedReadersObserver o) {
       }
     }
     replaceStoreFiles(filesToCompact, sfs, true);
+    storeEngine.resetCompactionWriter();

Review comment:
       Leave a big fat comment here as this is critical to the correctness of the BrokenStoreFileCleanerChore.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getCompactedfiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  private boolean isActiveStorefile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getStorefiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  boolean validate(Path file) {
+    if (HFileLink.isBackReferencesDir(file) || HFileLink.isBackReferencesDir(file.getParent())) {
+      return true;
+    }
+    return StoreFileInfo.validateStoreFileName(file.getName());
+  }
+
+  boolean isOldEnough(FileStatus file){
+    return file.getModificationTime() + ttl < System.currentTimeMillis();

Review comment:
       ```suggestion
       return file.getModificationTime() + ttl < EnvironmentEdgeManager.currentTime();
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;

Review comment:
       nit: `fileTtl`. It wasn't clear to me that this was the minimum age of the file before we'll actually clean it.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,13 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));

Review comment:
       This is a straight-up codebug, right? If we happen to have a non-null writer here, what's the implication on correctness?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       I didn't see the original implementation, but agree with Duo and Wellington that the current state seems reasonable.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");

Review comment:
       How about a test to validate that the TTL works?

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testSkippningCompactedFiles");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //run major compaction to generate compaced files
+    region.compact(true);
+
+    //make sure there are compacted files
+    HStore store = region.getStore(fam);
+    int compactedFiles =  store.getCompactedFilesCount();
+    assertTrue(compactedFiles > 0);
+
+    cleaner.chore();
+
+    //verify none of the compacted files wee deleted

Review comment:
       ```suggestion
       //verify none of the compacted files were deleted
   ```

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {

Review comment:
       ```suggestion
     public void testSkippingCompactedFiles() throws 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: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-961257613






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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-964164413


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  6s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 19s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   9m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   9m 56s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 18s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  3s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  3s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 226m 19s |  hbase-server in the patch failed.  |
   |  |   | 259m 50s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9e385a985cb9 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / a0ec946876 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/testReport/ |
   | Max. process+thread count | 3347 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r735736298



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       You are right, there is a slight gap between setting the writer to null and replacing the storefile list. Thanks for pointing it out.
   I would prefer to put the writer reset call to `HStore` in `doCompaction`, just after the `replaceStoreFiles` call. It would have the downside that StoreEngine has to expose the reset method, but I would argue moving it any lower on the call chain would just involve additional complications ( being called during `replayCompactionMarker`, handling different StoreEngine implementations )




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r735493337



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -1895,6 +1900,22 @@ private void initializeThreads() {
       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
           onlyMetaRefresh, this, this);
     }
+
+    int fileBasedStoreFileCleanerPeriod  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_PERIOD,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD);
+    int fileBasedStoreFileCleanerDelay  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY);
+    double fileBasedStoreFileCleanerDelayJitter = conf.getDouble(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY_JITTER,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER);
+    double jitterRate = (RandomUtils.nextDouble() - 0.5D) * fileBasedStoreFileCleanerDelayJitter;
+    long jitterValue = Math.round(fileBasedStoreFileCleanerDelay * jitterRate);

Review comment:
       Can we move all these properties initialisation to the Cleaner constructor itself?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,14 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));
+        writer = null;

Review comment:
       Do we need this? We are setting the writer in the next line...

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(FileBasedStoreFileCleaner.class);
+  public static final String FILEBASED_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.filebased.storefilecleaner.enabled";
+  public static final boolean DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED = false;
+  public static final String FILEBASED_STOREFILE_CLEANER_TTL =
+      "hbase.region.filebased.storefilecleaner.ttl";
+  public static final long DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY =
+      "hbase.region.filebased.storefilecleaner.delay";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.filebased.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String FILEBASED_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.filebased.storefilecleaner.period";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public FileBasedStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("FileBasedStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(FILEBASED_STOREFILE_CLEANER_ENABLED, DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(FILEBASED_STOREFILE_CLEANER_TTL, DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only clean do cleanup in store using file based storefile tracking
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "FileBasedStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("File based storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {

Review comment:
       >I suppose we should move the below isXXXFile to HRegion or HStore, with the protection of some locks. Otherwise we may have race, or at least, we depend on a very flaky order of the testing of each condition.
   
   I guess the `isOldEnough` check would prevent that problem, as we would only care about files older than `DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL`?
   
   > And do we really need to test isCompactingFile here? The compactedFile will not be tracked but compacting files should always be tracked?
   
   Yeah, this seems redundant with the `isActiveStorefile` check. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       > But I think we need to make sure that we only set writer to null after we finish committing these files to store file tracker
   
   Maybe we could move the call to `StoreFileTracker.replace` to the end of this class `compact` method, just before nulling the writer?




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-949875852


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 25s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 55s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m 11s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   9m 38s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 47s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 37s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 10s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |  10m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 47s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 244m 52s |  hbase-server in the patch failed.  |
   |  |   | 281m 42s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux ef0d0f909d6f 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 29fb33bf87 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/testReport/ |
   | Max. process+thread count | 3159 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r742662383



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
##########
@@ -110,7 +110,7 @@ public void init(StoreScanner sourceScanner, WriterFactory factory) {
     return paths;
   }
 
-  protected abstract Collection<StoreFileWriter> writers();
+  public abstract Collection<StoreFileWriter> writers();

Review comment:
       Thanks for pointing it out.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-961283009


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |  10m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |  11m 46s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |  11m 35s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server generated 5 new + 21 unchanged - 0 fixed = 26 total (was 21)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 167m 52s |  hbase-server in the patch failed.  |
   |  |   | 206m 24s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c31a3e1d5a27 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 3240b4b39c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/testReport/ |
   | Max. process+thread count | 3543 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r743324474



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){

Review comment:
       The 'compacting file' has its meaning in HBase. There is a filesCompacting field in HStore class, which is the files being compacted currently, not the files written out. So here let's change the name, otherwise it will confuse people.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
##########
@@ -79,5 +79,6 @@ protected void abortWriter(T writer) throws IOException {
           e);
       }
     }
+    writer = null;

Review comment:
       Do we still need to set writer to null in this method? Not in the reset method?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private

Review comment:
       Do we need this extra IA annotation? The class itself is IA.Private.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {

Review comment:
       Let's at least add some comments here to mention that, the compacted files aer the files which may still be referenced by some readers so we can not use it, see the javadoc for StoreFileManager.getCompactedfiles for more details. Otherwise it will be a bit strange to developers...




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-963689036


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 34s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   9m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   9m 59s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 16s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  3s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  3s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  2s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server generated 5 new + 21 unchanged - 0 fixed = 26 total (was 21)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 218m 56s |  hbase-server in the patch passed.  |
   |  |   | 252m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c01de929f3c1 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / a0ec946876 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/testReport/ |
   | Max. process+thread count | 3194 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-949852556


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  2s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 23s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   9m 14s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m  6s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 20s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 20s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 212m  9s |  hbase-server in the patch passed.  |
   |  |   | 248m  0s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0338de158941 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 29fb33bf87 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/testReport/ |
   | Max. process+thread count | 3383 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r742666552



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {

Review comment:
       IDEA formatter seems to prefer the single line format, so there might be a few of these in the code. I tend to change then back when I notice.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736329025



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       > I would prefer to put the writer reset call to HStore in doCompaction, just after the replaceStoreFiles call.
   
   So move `commitWriter` call out of `Compactor.compact`? Or do you plan to move the nulling of write to a `resetWriter` method?
   
   >  It would have the downside that StoreEngine has to expose the reset method, but I would argue moving it any lower on the call chain would just involve additional complications ( being called during replayCompactionMarker, handling different StoreEngine implementations )
   
   Yeah, looks like your suggestion is less complex, and you have an idea of the impacts, so I  just trust your judgement.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] joshelser commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
joshelser commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-961277961


   Just to make my intent clear, I think with a test method to exercise the TTL effectiveness and the minor code-formatting cleanup, this is fine to commit.
   
   Not sure if @Apache9 has more opinions on this change before he'd like to see it merged to the feature branch.


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r744817192



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
##########
@@ -79,5 +79,6 @@ protected void abortWriter(T writer) throws IOException {
           e);
       }
     }
+    writer = null;

Review comment:
       The reset method was introduced to allow resetting the writer from outside of a compactor, when the new files gets committed as storefiles. We also have to reset it in every abortWriter, but since these methods have direct access to the variable I would prefer to do it like this instead of calling the reset method.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r743324474



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){

Review comment:
       The 'compacting file' has its meaning in HBase. There is a filesCompacting field in HStore class, which is the files being compacted currently, not the files written out. So here let's change the name, otherwise it will confuse people.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
##########
@@ -79,5 +79,6 @@ protected void abortWriter(T writer) throws IOException {
           e);
       }
     }
+    writer = null;

Review comment:
       Do we still need to set writer to null in this method? Not in the reset method?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private

Review comment:
       Do we need this extra IA annotation? The class itself is IA.Private.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {

Review comment:
       Let's at least add some comments here to mention that, the compacted files aer the files which may still be referenced by some readers so we can not use it, see the javadoc for StoreFileManager.getCompactedfiles for more details. Otherwise it will be a bit strange to developers...




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-964107684


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 17s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m  6s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m 27s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 26s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 47s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   9m 25s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 14s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 41s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 150m 47s |  hbase-server in the patch passed.  |
   |  |   | 184m 56s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7ecdc4fad9ee 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / a0ec946876 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/testReport/ |
   | Max. process+thread count | 3982 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-954004625


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 15s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m 17s |  HBASE-26067 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  HBASE-26067 passed  |
   | +1 :green_heart: |  spotbugs  |   2m 22s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   2m 31s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 32s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 32s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 20s |  hbase-server: The patch generated 21 new + 62 unchanged - 0 fixed = 83 total (was 62)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  22m 15s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 20s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 13s |  The patch does not generate ASF License warnings.  |
   |  |   |  54m 19s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 5ec5a3338182 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 3240b4b39c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 96 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r735746314



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -1895,6 +1900,22 @@ private void initializeThreads() {
       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
           onlyMetaRefresh, this, this);
     }
+
+    int fileBasedStoreFileCleanerPeriod  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_PERIOD,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD);
+    int fileBasedStoreFileCleanerDelay  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY);
+    double fileBasedStoreFileCleanerDelayJitter = conf.getDouble(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY_JITTER,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER);
+    double jitterRate = (RandomUtils.nextDouble() - 0.5D) * fileBasedStoreFileCleanerDelayJitter;
+    long jitterValue = Math.round(fileBasedStoreFileCleanerDelay * jitterRate);

Review comment:
       I only put it there because StorefileRefresherChore have done the same. But I could move it if you would like to.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r735718289



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
##########
@@ -176,7 +176,7 @@ public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) th
    * Whether the implementation of this tracker requires you to write to temp directory first, i.e,
    * does not allow broken store files under the actual data directory.
    */
-  protected abstract boolean requireWritingToTmpDirFirst();
+  public abstract boolean requireWritingToTmpDirFirst();

Review comment:
       Thanks I forgot to clean this up.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r744817878



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {

Review comment:
       I'll add a short. expalnation




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r744817192



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
##########
@@ -79,5 +79,6 @@ protected void abortWriter(T writer) throws IOException {
           e);
       }
     }
+    writer = null;

Review comment:
       The reset method was introduced to allow resetting the writer from outside of a compactor, when the new files gets committed as storefiles. We also have reset it in every abortWriter, but since these methods have direct access to the variable I would prefer to do it like this instead of calling the reset method.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r744858544



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
##########
@@ -79,5 +79,6 @@ protected void abortWriter(T writer) throws IOException {
           e);
       }
     }
+    writer = null;

Review comment:
       It is just a bit weird that we set writer to null directly in these methods while we do not in some other methods. Please add some comments to explain the reason, that we will only call reset from upper layer when we successfully commit the writers so if we fail to commit the writers, we need to reset it by our own.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-961283009


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |  10m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |  11m 46s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 19s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |  11m 35s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server generated 5 new + 21 unchanged - 0 fixed = 26 total (was 21)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 167m 52s |  hbase-server in the patch failed.  |
   |  |   | 206m 24s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c31a3e1d5a27 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 3240b4b39c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/testReport/ |
   | Max. process+thread count | 3543 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-954164774


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 11s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 54s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m 33s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |  10m  1s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |  11m  5s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 38s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 34s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 34s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m 55s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 50s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 239m 57s |  hbase-server in the patch failed.  |
   |  |   | 279m 25s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0cb6e6a95b9b 4.15.0-142-generic #146-Ubuntu SMP Tue Apr 13 01:11:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 3240b4b39c |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/testReport/ |
   | Max. process+thread count | 3362 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] joshelser commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
joshelser commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-961277961


   Just to make my intent clear, I think with a test method to exercise the TTL effectiveness and the minor code-formatting cleanup, this is fine to commit.
   
   Not sure if @Apache9 has more opinions on this change before he'd like to see it merged to the feature branch.


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] joshelser commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r743071792



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,13 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));

Review comment:
       > the worst case scenario is that the cleaner would skip this file once because it would assume the compaction is still ongoing
   
   Cool, yeah, agree that is not the end of the world. `LOG.warn` seems appropriate :)




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 merged pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 merged pull request #3786:
URL: https://github.com/apache/hbase/pull/3786


   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r742685268



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,13 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));

Review comment:
       That I missed a point where writer reset should have been added or code was changed that allowed this to happen. As far as I can tell this can not happen now and it would be harmless even if it could happen. The worst case scenario is that the cleaner would skip this file once because it would assume the compaction is still ongoing. But it felt wrong not to log a warn before overwriting the writer.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r744806941



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){

Review comment:
       I'm renaming it to "isCompactionResultFile"




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-963512850


   @Apache9 Rebased my changes


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-964005325


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 30s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m 46s |  HBASE-26067 passed  |
   | +1 :green_heart: |  checkstyle  |   1m 12s |  HBASE-26067 passed  |
   | +1 :green_heart: |  spotbugs  |   2m 18s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   2m 28s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 18s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 18s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m  9s |  hbase-server: The patch generated 0 new + 61 unchanged - 1 fixed = 61 total (was 62)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 33s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 13s |  The patch does not generate ASF License warnings.  |
   |  |   |  50m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 0de76c87344c 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / a0ec946876 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | Max. process+thread count | 95 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/6/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r738440217



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       I'm ok with that.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] joshelser commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r742417329



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {

Review comment:
       nit: javadoc

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {

Review comment:
       ```suggestion
   @InterfaceAudience.Private
   public class BrokenStoreFileCleaner extends ScheduledChore {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
##########
@@ -72,13 +72,20 @@ public StoreFileWriter createWriter(InternalScanner scanner,
   }
 
   @Override
+  protected void abortWriter() throws IOException {
+    abortWriter(writer);
+  }
+
   protected void abortWriter(StoreFileWriter writer) throws IOException {
     Path leftoverFile = writer.getPath();
     try {
       writer.close();
     } catch (IOException e) {
       LOG.warn("Failed to close the writer after an unfinished compaction.", e);
     }
+    finally {

Review comment:
       ```suggestion
       } finally {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {
+    return storeFileTracker.requireWritingToTmpDirFirst();
+  }
+
+  public void resetCompactionWriter(){

Review comment:
       And a javadoc comment here as this is critical to ensuring that we know which files are sane to be deleted, please. Touch on `doCompaction` calling this.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +546,24 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){
+      return Collections.emptyList();
+    }
+    synchronized (writer){
+      if (writer instanceof StoreFileWriter){
+        return Arrays.asList(((StoreFileWriter)writer).getPath());
+      }
+      return ((AbstractMultiFileWriter)writer).writers().stream().map(sfw -> sfw.getPath()).collect(
+        Collectors.toList());
+    }
+  }
+
+  /**
+   * Reset the Writer when the new storefiles were successfully added
+   */
+  public  void resetWriter(){

Review comment:
       ```suggestion
     public void resetWriter() {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
##########
@@ -110,7 +110,7 @@ public void init(StoreScanner sourceScanner, WriterFactory factory) {
     return paths;
   }
 
-  protected abstract Collection<StoreFileWriter> writers();
+  public abstract Collection<StoreFileWriter> writers();

Review comment:
       nit: should have Javadoc on a public method.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -1159,6 +1159,8 @@ public void deleteChangedReaderObserver(ChangedReadersObserver o) {
       }
     }
     replaceStoreFiles(filesToCompact, sfs, true);
+    storeEngine.resetCompactionWriter();

Review comment:
       Leave a big fat comment here as this is critical to the correctness of the BrokenStoreFileCleanerChore.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getCompactedfiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  private boolean isActiveStorefile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getStorefiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  boolean validate(Path file) {
+    if (HFileLink.isBackReferencesDir(file) || HFileLink.isBackReferencesDir(file.getParent())) {
+      return true;
+    }
+    return StoreFileInfo.validateStoreFileName(file.getName());
+  }
+
+  boolean isOldEnough(FileStatus file){
+    return file.getModificationTime() + ttl < System.currentTimeMillis();

Review comment:
       ```suggestion
       return file.getModificationTime() + ttl < EnvironmentEdgeManager.currentTime();
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;

Review comment:
       nit: `fileTtl`. It wasn't clear to me that this was the minimum age of the file before we'll actually clean it.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,13 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));

Review comment:
       This is a straight-up codebug, right? If we happen to have a non-null writer here, what's the implication on correctness?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       I didn't see the original implementation, but agree with Duo and Wellington that the current state seems reasonable.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");

Review comment:
       How about a test to validate that the TTL works?

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testSkippningCompactedFiles");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //run major compaction to generate compaced files
+    region.compact(true);
+
+    //make sure there are compacted files
+    HStore store = region.getStore(fam);
+    int compactedFiles =  store.getCompactedFilesCount();
+    assertTrue(compactedFiles > 0);
+
+    cleaner.chore();
+
+    //verify none of the compacted files wee deleted

Review comment:
       ```suggestion
       //verify none of the compacted files were deleted
   ```

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {

Review comment:
       ```suggestion
     public void testSkippingCompactedFiles() throws 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: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r735122599



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
##########
@@ -176,7 +176,7 @@ public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) th
    * Whether the implementation of this tracker requires you to write to temp directory first, i.e,
    * does not allow broken store files under the actual data directory.
    */
-  protected abstract boolean requireWritingToTmpDirFirst();
+  public abstract boolean requireWritingToTmpDirFirst();

Review comment:
       Just remove this line if we move this method up to StoreFileTracker.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
##########
@@ -88,4 +88,10 @@ void replace(Collection<StoreFileInfo> compactedFiles, Collection<StoreFileInfo>
    * @param builder The table descriptor builder for the given table.
    */
   TableDescriptorBuilder updateWithTrackerConfigs(TableDescriptorBuilder builder);
+
+  /**
+   * Whether the implementation of this tracker requires you to write to temp directory first, i.e,
+   * does not allow broken store files under the actual data directory.
+   */
+  public boolean requireWritingToTmpDirFirst();

Review comment:
       nits: StoreFileTracker is an interface so we do not need the public modifier here.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       OK, so this is the trick here, in general I think the approach is fine, after committing we will set writer to null. But I think we need to make sure that we only set writer to null after we finish committing these files to store file tracker. In the current implementation, I do not think it is the case, at least in the below DateTieredCompactor, we set writer to null directly after we commit(it is just a close actually) the writers, we will then use the returned files to update the store file tracker.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {

Review comment:
       Let's not name it 'FileBased'. I think it should be more general, maybe 'BrokenStoreFileCleaner'.
   
   For now, the only condition is whether we will write to the data directory directly. No matter what is the actual store file tracker implementation, if it writes to the data directory directly, then we need this cleaner, otherwise we do not.
   
   

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(FileBasedStoreFileCleaner.class);
+  public static final String FILEBASED_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.filebased.storefilecleaner.enabled";
+  public static final boolean DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED = false;
+  public static final String FILEBASED_STOREFILE_CLEANER_TTL =
+      "hbase.region.filebased.storefilecleaner.ttl";
+  public static final long DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY =
+      "hbase.region.filebased.storefilecleaner.delay";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.filebased.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String FILEBASED_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.filebased.storefilecleaner.period";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public FileBasedStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("FileBasedStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(FILEBASED_STOREFILE_CLEANER_ENABLED, DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(FILEBASED_STOREFILE_CLEANER_TTL, DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only clean do cleanup in store using file based storefile tracking
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "FileBasedStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("File based storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {

Review comment:
       I suppose we should move the below isXXXFile to HRegion or HStore, with the protection of some locks. Otherwise we may have race, or at least, we depend on a very flaky order of the testing of each condition.
   
   And do we really need to test isCompactingFile here? The compactedFile will not be tracked but compacting files should always be tracked?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(FileBasedStoreFileCleaner.class);
+  public static final String FILEBASED_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.filebased.storefilecleaner.enabled";
+  public static final boolean DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED = false;
+  public static final String FILEBASED_STOREFILE_CLEANER_TTL =
+      "hbase.region.filebased.storefilecleaner.ttl";
+  public static final long DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY =
+      "hbase.region.filebased.storefilecleaner.delay";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.filebased.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String FILEBASED_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.filebased.storefilecleaner.period";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public FileBasedStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("FileBasedStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(FILEBASED_STOREFILE_CLEANER_ENABLED, DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(FILEBASED_STOREFILE_CLEANER_TTL, DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only clean do cleanup in store using file based storefile tracking
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {

Review comment:
       OK, good, this is exactly what I expected. So the only problem is the naming. Let's find a better 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: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r735748629



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,14 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));
+        writer = null;

Review comment:
       Setting the writer to null without any warning felt wrong. It's there to make sure if we run into a corner case where the writer was not reset we at least get a warning and could later investigate.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r735718786



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {

Review comment:
       Ok, I'm going with BrokenStoreFileCleaner then.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r744817878



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {

Review comment:
       I'll add a short. explanation

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {

Review comment:
       I'll add a short explanation




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-961257613


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 41s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m 15s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   9m 18s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 24s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 143m  1s |  hbase-server in the patch passed.  |
   |  |   | 175m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b840afe10d42 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 3240b4b39c |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/testReport/ |
   | Max. process+thread count | 3767 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r743324474



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){

Review comment:
       The 'compacting file' has its meaning in HBase. There is a filesCompacting field in HStore class, which is the files being compacted currently, not the files written out. So here let's change the name, otherwise it will confuse people.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
##########
@@ -79,5 +79,6 @@ protected void abortWriter(T writer) throws IOException {
           e);
       }
     }
+    writer = null;

Review comment:
       Do we still need to set writer to null in this method? Not in the reset method?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private

Review comment:
       Do we need this extra IA annotation? The class itself is IA.Private.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {

Review comment:
       Let's at least add some comments here to mention that, the compacted files aer the files which may still be referenced by some readers so we can not use it, see the javadoc for StoreFileManager.getCompactedfiles for more details. Otherwise it will be a bit strange to developers...




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r744806633



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private

Review comment:
       You are right, deleting it.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] joshelser commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r742417329



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {

Review comment:
       nit: javadoc

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {

Review comment:
       ```suggestion
   @InterfaceAudience.Private
   public class BrokenStoreFileCleaner extends ScheduledChore {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
##########
@@ -72,13 +72,20 @@ public StoreFileWriter createWriter(InternalScanner scanner,
   }
 
   @Override
+  protected void abortWriter() throws IOException {
+    abortWriter(writer);
+  }
+
   protected void abortWriter(StoreFileWriter writer) throws IOException {
     Path leftoverFile = writer.getPath();
     try {
       writer.close();
     } catch (IOException e) {
       LOG.warn("Failed to close the writer after an unfinished compaction.", e);
     }
+    finally {

Review comment:
       ```suggestion
       } finally {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {
+    return storeFileTracker.requireWritingToTmpDirFirst();
+  }
+
+  public void resetCompactionWriter(){

Review comment:
       And a javadoc comment here as this is critical to ensuring that we know which files are sane to be deleted, please. Touch on `doCompaction` calling this.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +546,24 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){
+      return Collections.emptyList();
+    }
+    synchronized (writer){
+      if (writer instanceof StoreFileWriter){
+        return Arrays.asList(((StoreFileWriter)writer).getPath());
+      }
+      return ((AbstractMultiFileWriter)writer).writers().stream().map(sfw -> sfw.getPath()).collect(
+        Collectors.toList());
+    }
+  }
+
+  /**
+   * Reset the Writer when the new storefiles were successfully added
+   */
+  public  void resetWriter(){

Review comment:
       ```suggestion
     public void resetWriter() {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
##########
@@ -110,7 +110,7 @@ public void init(StoreScanner sourceScanner, WriterFactory factory) {
     return paths;
   }
 
-  protected abstract Collection<StoreFileWriter> writers();
+  public abstract Collection<StoreFileWriter> writers();

Review comment:
       nit: should have Javadoc on a public method.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -1159,6 +1159,8 @@ public void deleteChangedReaderObserver(ChangedReadersObserver o) {
       }
     }
     replaceStoreFiles(filesToCompact, sfs, true);
+    storeEngine.resetCompactionWriter();

Review comment:
       Leave a big fat comment here as this is critical to the correctness of the BrokenStoreFileCleanerChore.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getCompactedfiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  private boolean isActiveStorefile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getStorefiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  boolean validate(Path file) {
+    if (HFileLink.isBackReferencesDir(file) || HFileLink.isBackReferencesDir(file.getParent())) {
+      return true;
+    }
+    return StoreFileInfo.validateStoreFileName(file.getName());
+  }
+
+  boolean isOldEnough(FileStatus file){
+    return file.getModificationTime() + ttl < System.currentTimeMillis();

Review comment:
       ```suggestion
       return file.getModificationTime() + ttl < EnvironmentEdgeManager.currentTime();
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;

Review comment:
       nit: `fileTtl`. It wasn't clear to me that this was the minimum age of the file before we'll actually clean it.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,13 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));

Review comment:
       This is a straight-up codebug, right? If we happen to have a non-null writer here, what's the implication on correctness?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       I didn't see the original implementation, but agree with Duo and Wellington that the current state seems reasonable.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");

Review comment:
       How about a test to validate that the TTL works?

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testSkippningCompactedFiles");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //run major compaction to generate compaced files
+    region.compact(true);
+
+    //make sure there are compacted files
+    HStore store = region.getStore(fam);
+    int compactedFiles =  store.getCompactedFilesCount();
+    assertTrue(compactedFiles > 0);
+
+    cleaner.chore();
+
+    //verify none of the compacted files wee deleted

Review comment:
       ```suggestion
       //verify none of the compacted files were deleted
   ```

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {

Review comment:
       ```suggestion
     public void testSkippingCompactedFiles() throws Exception {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {

Review comment:
       nit: javadoc

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {

Review comment:
       ```suggestion
   @InterfaceAudience.Private
   public class BrokenStoreFileCleaner extends ScheduledChore {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
##########
@@ -72,13 +72,20 @@ public StoreFileWriter createWriter(InternalScanner scanner,
   }
 
   @Override
+  protected void abortWriter() throws IOException {
+    abortWriter(writer);
+  }
+
   protected void abortWriter(StoreFileWriter writer) throws IOException {
     Path leftoverFile = writer.getPath();
     try {
       writer.close();
     } catch (IOException e) {
       LOG.warn("Failed to close the writer after an unfinished compaction.", e);
     }
+    finally {

Review comment:
       ```suggestion
       } finally {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
##########
@@ -514,4 +514,12 @@ public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  public boolean requireWritingToTmpDirFirst() {
+    return storeFileTracker.requireWritingToTmpDirFirst();
+  }
+
+  public void resetCompactionWriter(){

Review comment:
       And a javadoc comment here as this is critical to ensuring that we know which files are sane to be deleted, please. Touch on `doCompaction` calling this.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +546,24 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){
+      return Collections.emptyList();
+    }
+    synchronized (writer){
+      if (writer instanceof StoreFileWriter){
+        return Arrays.asList(((StoreFileWriter)writer).getPath());
+      }
+      return ((AbstractMultiFileWriter)writer).writers().stream().map(sfw -> sfw.getPath()).collect(
+        Collectors.toList());
+    }
+  }
+
+  /**
+   * Reset the Writer when the new storefiles were successfully added
+   */
+  public  void resetWriter(){

Review comment:
       ```suggestion
     public void resetWriter() {
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
##########
@@ -110,7 +110,7 @@ public void init(StoreScanner sourceScanner, WriterFactory factory) {
     return paths;
   }
 
-  protected abstract Collection<StoreFileWriter> writers();
+  public abstract Collection<StoreFileWriter> writers();

Review comment:
       nit: should have Javadoc on a public method.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -1159,6 +1159,8 @@ public void deleteChangedReaderObserver(ChangedReadersObserver o) {
       }
     }
     replaceStoreFiles(filesToCompact, sfs, true);
+    storeEngine.resetCompactionWriter();

Review comment:
       Leave a big fat comment here as this is critical to the correctness of the BrokenStoreFileCleanerChore.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getCompactedfiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  private boolean isActiveStorefile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getStorefiles().stream().anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  boolean validate(Path file) {
+    if (HFileLink.isBackReferencesDir(file) || HFileLink.isBackReferencesDir(file.getParent())) {
+      return true;
+    }
+    return StoreFileInfo.validateStoreFileName(file.getName());
+  }
+
+  boolean isOldEnough(FileStatus file){
+    return file.getModificationTime() + ttl < System.currentTimeMillis();

Review comment:
       ```suggestion
       return file.getModificationTime() + ttl < EnvironmentEdgeManager.currentTime();
   ```

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;

Review comment:
       nit: `fileTtl`. It wasn't clear to me that this was the minimum age of the file before we'll actually clean it.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,13 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));

Review comment:
       This is a straight-up codebug, right? If we happen to have a non-null writer here, what's the implication on correctness?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -537,4 +547,17 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){

Review comment:
       I didn't see the original implementation, but agree with Duo and Wellington that the current state seems reasonable.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");

Review comment:
       How about a test to validate that the TTL works?

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testSkippningCompactedFiles");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //run major compaction to generate compaced files
+    region.compact(true);
+
+    //make sure there are compacted files
+    HStore store = region.getStore(fam);
+    int compactedFiles =  store.getCompactedFilesCount();
+    assertTrue(compactedFiles > 0);
+
+    cleaner.chore();
+
+    //verify none of the compacted files wee deleted

Review comment:
       ```suggestion
       //verify none of the compacted files were deleted
   ```

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import java.io.IOException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster().getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippningCompactedFiles() throws Exception {

Review comment:
       ```suggestion
     public void testSkippingCompactedFiles() throws 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: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r738419886



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private public class FileBasedStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(FileBasedStoreFileCleaner.class);
+  public static final String FILEBASED_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.filebased.storefilecleaner.enabled";
+  public static final boolean DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED = false;
+  public static final String FILEBASED_STOREFILE_CLEANER_TTL =
+      "hbase.region.filebased.storefilecleaner.ttl";
+  public static final long DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY =
+      "hbase.region.filebased.storefilecleaner.delay";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String FILEBASED_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.filebased.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String FILEBASED_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.filebased.storefilecleaner.period";
+  public static final int DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long ttl;
+
+  public FileBasedStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("FileBasedStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(FILEBASED_STOREFILE_CLEANER_ENABLED, DEFAULT_FILEBASED_STOREFILE_CLEANER_ENABLED));
+    ttl = conf.getLong(FILEBASED_STOREFILE_CLEANER_TTL, DEFAULT_FILEBASED_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only clean do cleanup in store using file based storefile tracking
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "FileBasedStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("File based storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {

Review comment:
       >The confusion might come from suboptimal naming.
   isActiveStorefile -> lists the currently active storefiles
   isCompactedFile -> are hfiles that got compacted, no longer active storefiles, but were not deleted yet (deletion is handled by a separate subsystem, so we should not touch them)
   isCompactingFile -> file(s) a currently running compaction is writing into. This will become the new storefile when the compaction is done. It is checked to make sure we do not break stuck/longrunning compactions even is they are stuck/idle for more than the configured TTL.
   I think we do need all of these checks.
   
   Thanks for clarifying. Thinking again, maybe it's worth having this extra check, for the exceptional cases compaction can last longer than the time threshold.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736641941



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -1895,6 +1900,22 @@ private void initializeThreads() {
       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
           onlyMetaRefresh, this, this);
     }
+
+    int fileBasedStoreFileCleanerPeriod  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_PERIOD,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_PERIOD);
+    int fileBasedStoreFileCleanerDelay  = conf.getInt(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY);
+    double fileBasedStoreFileCleanerDelayJitter = conf.getDouble(
+      FileBasedStoreFileCleaner.FILEBASED_STOREFILE_CLEANER_DELAY_JITTER,
+      FileBasedStoreFileCleaner.DEFAULT_FILEBASED_STOREFILE_CLEANER_DELAY_JITTER);
+    double jitterRate = (RandomUtils.nextDouble() - 0.5D) * fileBasedStoreFileCleanerDelayJitter;
+    long jitterValue = Math.round(fileBasedStoreFileCleanerDelay * jitterRate);

Review comment:
       After actually trying to move them to constructor I realized why it got implemented like this:
   These values are needed for the super class constructor. So the resulting constructor would look something like this:
   
   ```
     public BrokenStoreFileCleaner(final Stoppable stopper, Configuration conf,
       HRegionServer regionServer) {
       this((int) (conf.getInt(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY,
           BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY)
           + (RandomUtils.nextDouble() - 0.5D) * conf.getDouble(
           BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY_JITTER,
           BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER)),
         conf.getInt(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD,
           BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD), stopper, conf,
         regionServer);
     }
   ```
   Considering readability I would prefer to leave the config handling where it is. What do you think?
   




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] BukrosSzabolcs commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r736407220



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
##########
@@ -348,8 +353,14 @@ private InternalScanner postCompactScannerOpen(CompactionRequestImpl request, Sc
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));
+        writer = null;

Review comment:
       Oh, you are absolutely right.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache-HBase commented on pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#issuecomment-963646789


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 37s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 49s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 26s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  HBASE-26067 passed  |
   | -0 :warning: |  patch  |   9m 23s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 142m 23s |  hbase-server in the patch passed.  |
   |  |   | 176m 24s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3786 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 829bfc833fa7 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / a0ec946876 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/testReport/ |
   | Max. process+thread count | 3806 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3786/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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



[GitHub] [hbase] Apache9 commented on a change in pull request #3786: HBASE-26271: Cleanup the broken store files under data directory

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3786:
URL: https://github.com/apache/hbase/pull/3786#discussion_r745187340



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
##########
@@ -79,5 +79,6 @@ protected void abortWriter(T writer) throws IOException {
           e);
       }
     }
+    writer = null;

Review comment:
       Please add some comments here. Thanks.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
##########
@@ -0,0 +1,191 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper, Configuration conf,
+      HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @InterfaceAudience.Private
+  @Override public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles = Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactingFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactingFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  private boolean isCompactedFile(FileStatus file, HStore store) {

Review comment:
       Please add a simple comment or javadoc here? Thanks.




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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