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/13 18:05:57 UTC

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

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


   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
   Store chore metrics on RS and push them for aggregation to Master
   Add a new REST endpoint to get chore metrics from Master


-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
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:
       These changes allow me to list the paths of current compaction target files so I could make sure I do not delete the files currently being written even if the compaction is extremely slow.




-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
##########
@@ -63,22 +63,30 @@ public StoreFileWriter createWriter(InternalScanner scanner,
   }
 
   @Override
-  protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd,
+  protected List<Path> commitWriter(FileDetails fd,
       CompactionRequestImpl request) throws IOException {
     List<Path> newFiles = Lists.newArrayList(writer.getPath());
     writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles());
     writer.close();
+    writer = null;

Review comment:
       writer was refactored to be a class variable so I can list current compaction targets any time and can avoid deleting them. So the variable is re-used and have to be cleaned up after a compaction.




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

To unsubscribe, e-mail: 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   @Apache9 I was planning to cut up this PR into smaller, incremental PRs, starting with only the Cleaner as requested. I was planning to leave this PR here until all of those are done. But I would be happy to rebase this if you think those steps are not necessary. 


-- 
This is an automated message from the Apache Git Service.
To 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 closed pull request #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   


-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   @Apache9 rebased on top of HBASE-26067 and fixed a small issue in the REST model that worked on java8 but failed on 11.


-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
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:
       Why we need these changes in compactor?

##########
File path: hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
##########
@@ -92,6 +92,12 @@ public StorageClusterStatusResource getClusterStatusResource()
     return new StorageClusterStatusResource();
   }
 
+  @Path("status/fileBasedStoreFileCleaner")

Review comment:
       Also for the rest part, let's do these things in separated issues.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,200 @@
+/**
+ * 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:
       I think we could move the schedule logic into the SFT implementation? The implementation could choose whether to schedule a cleaner. And the only condition I could see now, is whether we will write to the data directory directly, if so, we need a cleaner, otherwise we do not. And the logic of the cleaner should be the same? Just list the data directory, compare the returned list with the one gotten from the SFT, if here are diffs then delete the unused files(of course we need to deal with some corner cases, maybe by timestamp, as discussed on jira).

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java
##########
@@ -161,6 +162,12 @@ default double getAverageLoad() {
    */
   Map<TableName, RegionStatesCount> getTableRegionStatesCount();
 
+  /**
+   * Provide information about FileBasedFileStoreCleaner chore
+   * @return
+   */
+  Map<String, FileBasedStoreFileCleanerStatus> getFileBasedStoreFileCleanerStatus();

Review comment:
       I think metrics could be a separated issue, which could make this PR smaller and let's focus on the core part first.




-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 21s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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 _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 41s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   7m 14s |  HBASE-26067 passed  |
   | +1 :green_heart: |  checkstyle  |   2m 21s |  HBASE-26067 passed  |
   | +1 :green_heart: |  spotbugs  |   9m 26s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 46s |  the patch passed  |
   | +1 :green_heart: |  cc  |   7m 46s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 32s |  hbase-hadoop-compat generated 6 new + 97 unchanged - 6 fixed = 103 total (was 103)  |
   | -0 :warning: |  javac  |   1m  4s |  hbase-client generated 1 new + 121 unchanged - 1 fixed = 122 total (was 122)  |
   | -0 :warning: |  checkstyle  |   0m 16s |  hbase-hadoop-compat: The patch generated 14 new + 0 unchanged - 0 fixed = 14 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 35s |  hbase-client: The patch generated 5 new + 181 unchanged - 0 fixed = 186 total (was 181)  |
   | -0 :warning: |  checkstyle  |   1m 19s |  hbase-server: The patch generated 27 new + 162 unchanged - 0 fixed = 189 total (was 162)  |
   | -0 :warning: |  checkstyle  |   0m 17s |  hbase-rest: The patch generated 3 new + 3 unchanged - 0 fixed = 6 total (was 3)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  21m 11s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 48s |  the patch passed  |
   | -1 :x: |  spotbugs  |   2m 28s |  hbase-server generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -1 :x: |  spotbugs  |   0m 54s |  hbase-rest generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | -1 :x: |  asflicense  |   0m 53s |  The patch generated 1 ASF License warnings.  |
   |  |   |  84m 52s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Dead store to fsStoreFiles in org.apache.hadoop.hbase.regionserver.FileBasedStoreFileCleaner.chore()  At FileBasedStoreFileCleaner.java:org.apache.hadoop.hbase.regionserver.FileBasedStoreFileCleaner.chore()  At FileBasedStoreFileCleaner.java:[line 94] |
   |  |  Random object created and used only once in org.apache.hadoop.hbase.regionserver.HRegionServer.initializeThreads()  At HRegionServer.java:only once in org.apache.hadoop.hbase.regionserver.HRegionServer.initializeThreads()  At HRegionServer.java:[line 1990] |
   | FindBugs | module:hbase-rest |
   |  |  Class org.apache.hadoop.hbase.rest.model.FileBasedStoreFileCleanerStatusModel defines non-transient non-serializable instance field fileBasedFileStoreCleanerStatus  In FileBasedStoreFileCleanerStatusModel.java:instance field fileBasedFileStoreCleanerStatus  In FileBasedStoreFileCleanerStatusModel.java |
   
   
   | 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-3751/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3751 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 31ea4f640230 4.15.0-147-generic #151-Ubuntu SMP Fri Jun 18 19:21:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 9c9789bbd2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/diff-compile-javac-hbase-hadoop-compat.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/diff-compile-javac-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-rest.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/new-spotbugs-hbase-rest.html |
   | asflicense | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-general-check/output/patch-asflicense-problems.txt |
   | Max. process+thread count | 87 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-hadoop-compat hbase-client hbase-server hbase-rest U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/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] wchevreuil commented on a change in pull request #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/FileBasedStoreFileCleanerStatusResource.java
##########
@@ -0,0 +1,74 @@
+/*
+ *
+ * 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.rest;
+
+import org.apache.hadoop.hbase.rest.model.FileBasedStoreFileCleanerStatusesModel;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import javax.ws.rs.GET;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.CacheControl;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.ResponseBuilder;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+
+@InterfaceAudience.Private
+public class FileBasedStoreFileCleanerStatusResource extends ResourceBase {

Review comment:
       Same as above for the FileBaseStoreFileCleaner.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
##########
@@ -85,4 +85,11 @@ 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.
+   */
+  boolean requireWritingToTmpDirFirst();

Review comment:
       Instead of exposing this, would it work if the file cleaner is placed on storefiletracker package?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,200 @@
+/**
+ * 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:
       Shouldn't we have an abstract StoreFileCleaner to concentrate common logic, and then for each SFT implementation specific, we would have an individual cleaner? Because we can't assume that FilaBasedStoreFileTracker would always be on.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
##########
@@ -63,22 +63,30 @@ public StoreFileWriter createWriter(InternalScanner scanner,
   }
 
   @Override
-  protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd,
+  protected List<Path> commitWriter(FileDetails fd,
       CompactionRequestImpl request) throws IOException {
     List<Path> newFiles = Lists.newArrayList(writer.getPath());
     writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles());
     writer.close();
+    writer = null;

Review comment:
       Why are we nulling this here? Shouldn't it throw IllegalArgumentException if a closed writer is tried to be used?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
##########
@@ -60,7 +60,7 @@ public MigrationStoreFileTracker(Configuration conf, boolean isPrimaryReplica, S
   }
 
   @Override
-  protected boolean requireWritingToTmpDirFirst() {
+  public boolean requireWritingToTmpDirFirst() {

Review comment:
       Same as above.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -1895,6 +1976,23 @@ 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);

Review comment:
       What if FileBasedStoreFileTracker isn't set?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
##########
@@ -95,7 +95,7 @@ public FileBasedStoreFileTracker(Configuration conf, boolean isPrimaryReplica, S
   }
 
   @Override
-  protected boolean requireWritingToTmpDirFirst() {
+  public boolean requireWritingToTmpDirFirst() {

Review comment:
       Does it make sense to set it to public here, since FileBasedStoreFileTracker is package private?




-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   OK, no problem, no need to rebase then. Let's start with smaller PRs.
   
   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



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

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
##########
@@ -95,7 +95,7 @@ public FileBasedStoreFileTracker(Configuration conf, boolean isPrimaryReplica, S
   }
 
   @Override
-  protected boolean requireWritingToTmpDirFirst() {
+  public boolean requireWritingToTmpDirFirst() {

Review comment:
       StoreEngine is in a different package. I needed a way to check if the cleaner should run for a given store and SFT did not expose anything I could work with. This seemed like the most convenient solution.




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

To unsubscribe, e-mail: 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  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 _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 34s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m 29s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 15s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  5s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 18s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 25s |  hbase-client in the patch failed.  |
   | -1 :x: |  compile  |   0m 41s |  hbase-server in the patch failed.  |
   | -1 :x: |  compile  |   0m 25s |  hbase-rest in the patch failed.  |
   | -0 :warning: |  javac  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javac  |   0m 41s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 25s |  hbase-rest in the patch failed.  |
   | -1 :x: |  shadedjars  |   3m 45s |  patch has 22 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-rest in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 39s |  hbase-hadoop-compat in the patch passed.  |
   | -1 :x: |  unit  |   0m 25s |  hbase-client in the patch failed.  |
   | -1 :x: |  unit  |   0m 41s |  hbase-server in the patch failed.  |
   | -1 :x: |  unit  |   0m 24s |  hbase-rest in the patch failed.  |
   |  |   |  34m 11s |   |
   
   
   | 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-3751/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3751 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux bd42b88d2855 4.15.0-156-generic #163-Ubuntu SMP Thu Aug 19 23:31:58 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / 9c9789bbd2 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-client.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-rest.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-client.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-rest.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-rest.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-client.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-rest.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/testReport/ |
   | Max. process+thread count | 376 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-hadoop-compat hbase-client hbase-server hbase-rest U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
##########
@@ -85,4 +85,11 @@ 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.
+   */
+  boolean requireWritingToTmpDirFirst();

Review comment:
       SFT would still not be available outside the StoreEngine.




-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  0s |  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 _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   8m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 54s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 46s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 57s |  the patch passed  |
   | -1 :x: |  shadedjars  |   3m 48s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client generated 1 new + 3 unchanged - 0 fixed = 4 total (was 3)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 36s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 15s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 157m 32s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   3m 49s |  hbase-rest in the patch passed.  |
   |  |   | 198m 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-3751/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3751 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7b9f4013b826 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 / 9c9789bbd2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-client.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/1/testReport/ |
   | Max. process+thread count | 4573 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-hadoop-compat hbase-client hbase-server hbase-rest U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/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] Apache-HBase commented on pull request #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 13s |  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 _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 55s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m 43s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 33s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  8s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 44s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 44s |  the patch passed  |
   | -1 :x: |  shadedjars  |   4m  6s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  7s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  3s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 40s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 22s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 152m 36s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   4m 14s |  hbase-rest in the patch passed.  |
   |  |   | 201m 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-3751/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3751 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 92a15caab9de 4.15.0-156-generic #163-Ubuntu SMP Thu Aug 19 23:31:58 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / d3556bf41f |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/testReport/ |
   | Max. process+thread count | 4077 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-hadoop-compat hbase-client hbase-server hbase-rest U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/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] BukrosSzabolcs commented on a change in pull request #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,200 @@
+/**
+ * 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:
       @Apache9 That is exactly what I have done.




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

To unsubscribe, e-mail: 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,200 @@
+/**
+ * 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:
       I'm not sure if there is any common logic. The checks and the metrics are implementation specific, the configs have to be separate because we might have different requirements for different usecases and there is not much else. Also as far as I'm aware the other SFT implementations do not need a cleaner like this.




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

To unsubscribe, e-mail: 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,200 @@
+/**
+ * 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:
       I see 3 potential issues with moving the cleaner to SFT:
   
   - This could easily result in a few thousand Chores per RS, when the feature is globally enabled, compared to the single Chore per RS we would have now. Also there are multiple cases where an SFT is created outside the StoreEngine and those would also have their own Chores, potentially causing concurrency issues.
   - Compacted files are listed in StoreFileManager and cleaned up by other subsystems. The chore should not delete them. Moving the Chore to SFT would make accessing the compacted files list hard.
   - I made a few small changes to make the currently running compaction's target files available to make sure we do not delete them even if the compaction is extremely slow. Moving the Chore to SFT would make accessing them hard.
   
   What do you think are the benefits of moving the cleaner to SFT?




-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  5s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-26067 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   3m  2s |  HBASE-26067 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 57s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  3s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  4s |  the patch passed  |
   | -1 :x: |  shadedjars  |   3m 47s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client generated 1 new + 3 unchanged - 0 fixed = 4 total (was 3)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 39s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 24s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 157m 47s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   4m 29s |  hbase-rest in the patch passed.  |
   |  |   | 199m  3s |   |
   
   
   | 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-3751/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3751 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux af04fe3e5953 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 / d3556bf41f |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-client.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/testReport/ |
   | Max. process+thread count | 4943 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-hadoop-compat hbase-client hbase-server hbase-rest U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/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] Apache9 commented on a change in pull request #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
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:
       So what you want to avoid here is for a long running compaction, the modification time of the file is already long ago but it is still being written, so if we only use timestamp to determine whether it is OK to delete it we may delete it incorrectly?
   This is a possible problem, but it still makes me a bit nervous that, we do not have any fencing here, so it is still possible that we hit some corner cases where we will delete these files?
   Please give me some time to think of this...
   
   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



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

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


   HBASE-26386 has been merged so you need to rebase here @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] Apache9 commented on pull request #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   Hi @BukrosSzabolcs , you need to rebase your PR as I have done a force push to HBASE-26067 to include HBASE-26348 so @GeorryHuang could implement HBASE-26263.
   
   Sorry for the inconvenience.


-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -1895,6 +1976,23 @@ 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);

Review comment:
       Then we skip that store. There is a check for each store on the rs.




-- 
This is an automated message from the Apache Git Service.
To 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FileBasedStoreFileCleaner.java
##########
@@ -0,0 +1,200 @@
+/**
+ * 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:
       Sounds reasonable. So it will be better to just let the chore to iterator over all the regions and stores to see if they need clean up, so we do not need to maintain the list of stores which needs to clean up(this would be a pain I suppose, as you need to hook up region open and close).
   So I think we could expose something like requireWritingToTmpFirst in the StoreEngine interface(we do not expose SFT to upper layer in HRegion and HStore), so you can implement this PR easier.
   WDYT?




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

To unsubscribe, e-mail: 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 #3751: HBASE-26271: Cleanup the broken store files under data directory

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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 _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 19s |  HBASE-26067 passed  |
   | +1 :green_heart: |  compile  |   6m 55s |  HBASE-26067 passed  |
   | +1 :green_heart: |  checkstyle  |   2m 20s |  HBASE-26067 passed  |
   | +1 :green_heart: |  spotbugs  |   8m 47s |  HBASE-26067 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 15s |  the patch passed  |
   | +1 :green_heart: |  compile  |   6m 59s |  the patch passed  |
   | +1 :green_heart: |  cc  |   6m 59s |  the patch passed  |
   | -0 :warning: |  javac  |   0m 28s |  hbase-hadoop-compat generated 6 new + 97 unchanged - 6 fixed = 103 total (was 103)  |
   | -0 :warning: |  javac  |   0m 55s |  hbase-client generated 1 new + 121 unchanged - 1 fixed = 122 total (was 122)  |
   | -0 :warning: |  checkstyle  |   0m 12s |  hbase-hadoop-compat: The patch generated 14 new + 0 unchanged - 0 fixed = 14 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 31s |  hbase-client: The patch generated 4 new + 181 unchanged - 0 fixed = 185 total (was 181)  |
   | -0 :warning: |  checkstyle  |   1m 14s |  hbase-server: The patch generated 27 new + 162 unchanged - 0 fixed = 189 total (was 162)  |
   | -0 :warning: |  checkstyle  |   0m 15s |  hbase-rest: The patch generated 9 new + 3 unchanged - 0 fixed = 12 total (was 3)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  21m 16s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 49s |  the patch passed  |
   | -1 :x: |  spotbugs  |   2m 24s |  hbase-server generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | -1 :x: |  asflicense  |   0m 53s |  The patch generated 2 ASF License warnings.  |
   |  |   |  82m 40s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Dead store to fsStoreFiles in org.apache.hadoop.hbase.regionserver.FileBasedStoreFileCleaner.chore()  At FileBasedStoreFileCleaner.java:org.apache.hadoop.hbase.regionserver.FileBasedStoreFileCleaner.chore()  At FileBasedStoreFileCleaner.java:[line 94] |
   |  |  Random object created and used only once in org.apache.hadoop.hbase.regionserver.HRegionServer.initializeThreads()  At HRegionServer.java:only once in org.apache.hadoop.hbase.regionserver.HRegionServer.initializeThreads()  At HRegionServer.java:[line 1990] |
   
   
   | 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-3751/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3751 |
   | JIRA Issue | HBASE-26271 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool |
   | uname | Linux 8758feec2c9d 4.15.0-147-generic #151-Ubuntu SMP Fri Jun 18 19:21:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-26067 / d3556bf41f |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/diff-compile-javac-hbase-hadoop-compat.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/diff-compile-javac-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-rest.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | asflicense | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/2/artifact/yetus-general-check/output/patch-asflicense-problems.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-hadoop-compat hbase-client hbase-server hbase-rest U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3751/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